Refactoring

This commit is contained in:
kssenii 2024-01-17 12:57:40 +01:00
parent 9146391e48
commit c905937ef6
32 changed files with 653 additions and 291 deletions

View File

@ -590,6 +590,7 @@
M(708, ILLEGAL_STATISTIC) \ M(708, ILLEGAL_STATISTIC) \
M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \ M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \
M(710, FAULT_INJECTED) \ M(710, FAULT_INJECTED) \
M(711, FILECACHE_ACCESS_DENIED) \
\ \
M(999, KEEPER_EXCEPTION) \ M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \ M(1000, POCO_EXCEPTION) \

View File

@ -48,6 +48,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
const String & source_file_path_, const String & source_file_path_,
const FileCache::Key & cache_key_, const FileCache::Key & cache_key_,
FileCachePtr cache_, FileCachePtr cache_,
const FileCacheUserInfo & user_,
ImplementationBufferCreator implementation_buffer_creator_, ImplementationBufferCreator implementation_buffer_creator_,
const ReadSettings & settings_, const ReadSettings & settings_,
const String & query_id_, const String & query_id_,
@ -70,6 +71,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
, implementation_buffer_creator(implementation_buffer_creator_) , implementation_buffer_creator(implementation_buffer_creator_)
, query_id(query_id_) , query_id(query_id_)
, current_buffer_id(getRandomASCIIString(8)) , current_buffer_id(getRandomASCIIString(8))
, user(user_)
, allow_seeks_after_first_read(allow_seeks_after_first_read_) , allow_seeks_after_first_read(allow_seeks_after_first_read_)
, use_external_buffer(use_external_buffer_) , use_external_buffer(use_external_buffer_)
, query_context_holder(cache_->getQueryContextHolder(query_id, settings_)) , query_context_holder(cache_->getQueryContextHolder(query_id, settings_))
@ -127,12 +129,12 @@ bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch()
if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache)
{ {
file_segments = cache->get(cache_key, file_offset_of_buffer_end, size, settings.filesystem_cache_segments_batch_size); file_segments = cache->get(cache_key, file_offset_of_buffer_end, size, settings.filesystem_cache_segments_batch_size, user.user_id);
} }
else else
{ {
CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); CreateFileSegmentSettings create_settings(FileSegmentKind::Regular);
file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size); file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user);
} }
return !file_segments->empty(); return !file_segments->empty();
} }
@ -166,7 +168,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm
{ {
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::CachedReadBufferCreateBufferMicroseconds); ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::CachedReadBufferCreateBufferMicroseconds);
auto path = file_segment.getPathInLocalCache(); auto path = file_segment.getPath();
if (cache_file_reader) if (cache_file_reader)
{ {
chassert(cache_file_reader->getFileName() == path); chassert(cache_file_reader->getFileName() == path);

View File

@ -1,12 +1,15 @@
#pragma once #pragma once
#include <Interpreters/Cache/FileCache.h> #include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/FileCache_fwd.h>
#include <Interpreters/Cache/QueryLimit.h>
#include <IO/SeekableReadBuffer.h> #include <IO/SeekableReadBuffer.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
#include <IO/ReadSettings.h> #include <IO/ReadSettings.h>
#include <IO/ReadBufferFromFileBase.h> #include <IO/ReadBufferFromFileBase.h>
#include <Interpreters/FilesystemCacheLog.h> #include <Interpreters/FilesystemCacheLog.h>
#include <Interpreters/Cache/FileSegment.h> #include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/Cache/UserInfo.h>
namespace CurrentMetrics namespace CurrentMetrics
@ -24,8 +27,9 @@ public:
CachedOnDiskReadBufferFromFile( CachedOnDiskReadBufferFromFile(
const String & source_file_path_, const String & source_file_path_,
const FileCache::Key & cache_key_, const FileCacheKey & cache_key_,
FileCachePtr cache_, FileCachePtr cache_,
const FileCacheUserInfo & user_,
ImplementationBufferCreator implementation_buffer_creator_, ImplementationBufferCreator implementation_buffer_creator_,
const ReadSettings & settings_, const ReadSettings & settings_,
const String & query_id_, const String & query_id_,
@ -102,7 +106,7 @@ private:
bool nextFileSegmentsBatch(); bool nextFileSegmentsBatch();
Poco::Logger * log; Poco::Logger * log;
FileCache::Key cache_key; FileCacheKey cache_key;
String source_file_path; String source_file_path;
FileCachePtr cache; FileCachePtr cache;
@ -145,13 +149,14 @@ private:
String query_id; String query_id;
String current_buffer_id; String current_buffer_id;
FileCacheUserInfo user;
bool allow_seeks_after_first_read; bool allow_seeks_after_first_read;
[[maybe_unused]]bool use_external_buffer; [[maybe_unused]]bool use_external_buffer;
CurrentMetrics::Increment metric_increment{CurrentMetrics::FilesystemCacheReadBuffers}; CurrentMetrics::Increment metric_increment{CurrentMetrics::FilesystemCacheReadBuffers};
ProfileEvents::Counters current_file_segment_counters; ProfileEvents::Counters current_file_segment_counters;
FileCache::QueryContextHolderPtr query_context_holder; FileCacheQueryLimit::QueryContextHolderPtr query_context_holder;
std::shared_ptr<FilesystemCacheLog> cache_log; std::shared_ptr<FilesystemCacheLog> cache_log;
}; };

View File

@ -1,10 +1,10 @@
#include "CachedOnDiskWriteBufferFromFile.h" #include "CachedOnDiskWriteBufferFromFile.h"
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/FilesystemCacheLog.h> #include <Interpreters/FilesystemCacheLog.h>
#include <Interpreters/Context.h>
#include <IO/SwapHelper.h> #include <IO/SwapHelper.h>
@ -25,6 +25,7 @@ namespace ErrorCodes
FileSegmentRangeWriter::FileSegmentRangeWriter( FileSegmentRangeWriter::FileSegmentRangeWriter(
FileCache * cache_, FileCache * cache_,
const FileSegment::Key & key_, const FileSegment::Key & key_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_, std::shared_ptr<FilesystemCacheLog> cache_log_,
const String & query_id_, const String & query_id_,
const String & source_path_) const String & source_path_)
@ -34,6 +35,7 @@ FileSegmentRangeWriter::FileSegmentRangeWriter(
, cache_log(cache_log_) , cache_log(cache_log_)
, query_id(query_id_) , query_id(query_id_)
, source_path(source_path_) , source_path(source_path_)
, user(user_)
{ {
} }
@ -148,7 +150,7 @@ FileSegment & FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSeg
/// We set max_file_segment_size to be downloaded, /// We set max_file_segment_size to be downloaded,
/// if we have less size to write, file segment will be resized in complete() method. /// if we have less size to write, file segment will be resized in complete() method.
file_segments = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings); file_segments = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings, user);
chassert(file_segments->size() == 1); chassert(file_segments->size() == 1);
return file_segments->front(); return file_segments->front();
} }
@ -193,7 +195,6 @@ void FileSegmentRangeWriter::completeFileSegment()
appendFilesystemCacheLog(file_segment); appendFilesystemCacheLog(file_segment);
} }
CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile( CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
std::unique_ptr<WriteBuffer> impl_, std::unique_ptr<WriteBuffer> impl_,
FileCachePtr cache_, FileCachePtr cache_,
@ -201,6 +202,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
const FileCache::Key & key_, const FileCache::Key & key_,
const String & query_id_, const String & query_id_,
const WriteSettings & settings_, const WriteSettings & settings_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_) std::shared_ptr<FilesystemCacheLog> cache_log_)
: WriteBufferFromFileDecorator(std::move(impl_)) : WriteBufferFromFileDecorator(std::move(impl_))
, log(&Poco::Logger::get("CachedOnDiskWriteBufferFromFile")) , log(&Poco::Logger::get("CachedOnDiskWriteBufferFromFile"))
@ -208,6 +210,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
, source_path(source_path_) , source_path(source_path_)
, key(key_) , key(key_)
, query_id(query_id_) , query_id(query_id_)
, user(user_)
, throw_on_error_from_cache(settings_.throw_on_error_from_cache) , throw_on_error_from_cache(settings_.throw_on_error_from_cache)
, cache_log(!query_id_.empty() && settings_.enable_filesystem_cache_log ? cache_log_ : nullptr) , cache_log(!query_id_.empty() && settings_.enable_filesystem_cache_log ? cache_log_ : nullptr)
{ {
@ -233,7 +236,7 @@ void CachedOnDiskWriteBufferFromFile::nextImpl()
{ {
/// If something was already written to cache, remove it. /// If something was already written to cache, remove it.
cache_writer.reset(); cache_writer.reset();
cache->removeKeyIfExists(key); cache->removeKeyIfExists(key, user.user_id);
throw; throw;
} }
@ -246,7 +249,7 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size, bool t
if (!cache_writer) if (!cache_writer)
{ {
cache_writer = std::make_unique<FileSegmentRangeWriter>(cache.get(), key, cache_log, query_id, source_path); cache_writer = std::make_unique<FileSegmentRangeWriter>(cache.get(), key, user, cache_log, query_id, source_path);
} }
Stopwatch watch(CLOCK_MONOTONIC); Stopwatch watch(CLOCK_MONOTONIC);

View File

@ -2,7 +2,9 @@
#include <IO/WriteBufferFromFileDecorator.h> #include <IO/WriteBufferFromFileDecorator.h>
#include <IO/WriteSettings.h> #include <IO/WriteSettings.h>
#include <Interpreters/Cache/FileCache.h> #include <Interpreters/Cache/FileCache_fwd.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/FilesystemCacheLog.h> #include <Interpreters/FilesystemCacheLog.h>
namespace Poco namespace Poco
@ -25,8 +27,12 @@ class FileSegmentRangeWriter
{ {
public: public:
FileSegmentRangeWriter( FileSegmentRangeWriter(
FileCache * cache_, const FileSegment::Key & key_, FileCache * cache_,
std::shared_ptr<FilesystemCacheLog> cache_log_, const String & query_id_, const String & source_path_); const FileSegment::Key & key_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_,
const String & query_id_,
const String & source_path_);
/** /**
* Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to * Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to
@ -52,6 +58,7 @@ private:
std::shared_ptr<FilesystemCacheLog> cache_log; std::shared_ptr<FilesystemCacheLog> cache_log;
String query_id; String query_id;
String source_path; String source_path;
FileCacheUserInfo user;
FileSegmentsHolderPtr file_segments; FileSegmentsHolderPtr file_segments;
@ -71,15 +78,18 @@ public:
std::unique_ptr<WriteBuffer> impl_, std::unique_ptr<WriteBuffer> impl_,
FileCachePtr cache_, FileCachePtr cache_,
const String & source_path_, const String & source_path_,
const FileCache::Key & key_, const FileCacheKey & key_,
const String & query_id_, const String & query_id_,
const WriteSettings & settings_, const WriteSettings & settings_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_); std::shared_ptr<FilesystemCacheLog> cache_log_);
void nextImpl() override; void nextImpl() override;
void finalizeImpl() override; void finalizeImpl() override;
bool cachingStopped() const { return cache_in_error_state_or_disabled; }
private: private:
void cacheData(char * data, size_t size, bool throw_on_error); void cacheData(char * data, size_t size, bool throw_on_error);
@ -87,10 +97,11 @@ private:
FileCachePtr cache; FileCachePtr cache;
String source_path; String source_path;
FileCache::Key key; FileCacheKey key;
size_t current_download_offset = 0; size_t current_download_offset = 0;
const String query_id; const String query_id;
const FileCacheUserInfo user;
bool throw_on_error_from_cache; bool throw_on_error_from_cache;
bool cache_in_error_state_or_disabled = false; bool cache_in_error_state_or_disabled = false;

View File

@ -82,6 +82,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
object_path, object_path,
cache_key, cache_key,
settings.remote_fs_cache, settings.remote_fs_cache,
FileCache::getCommonUser(),
std::move(current_read_buffer_creator), std::move(current_read_buffer_creator),
settings, settings,
query_id, query_id,

View File

@ -113,6 +113,7 @@ std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// N
key, key,
CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "", CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "",
modified_write_settings, modified_write_settings,
FileCache::getCommonUser(),
Context::getGlobalContextInstance()->getFilesystemCacheLog()); Context::getGlobalContextInstance()->getFilesystemCacheLog());
} }
@ -125,7 +126,7 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c
return; return;
/// Add try catch? /// Add try catch?
cache->removeKeyIfExists(getCacheKey(path_key_for_cache)); cache->removeKeyIfExists(getCacheKey(path_key_for_cache), FileCache::getCommonUser().user_id);
} }
void CachedObjectStorage::removeObject(const StoredObject & object) void CachedObjectStorage::removeObject(const StoredObject & object)

View File

@ -13,6 +13,7 @@
#include <base/hex.h> #include <base/hex.h>
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
#include <Common/ElapsedTimeProfileEventIncrement.h> #include <Common/ElapsedTimeProfileEventIncrement.h>
#include <Core/ServerUUID.h>
#include <filesystem> #include <filesystem>
@ -28,29 +29,35 @@ namespace ProfileEvents
extern const Event FilesystemCacheGetMicroseconds; extern const Event FilesystemCacheGetMicroseconds;
} }
namespace
{
size_t roundDownToMultiple(size_t num, size_t multiple)
{
return (num / multiple) * multiple;
}
size_t roundUpToMultiple(size_t num, size_t multiple)
{
return roundDownToMultiple(num + multiple - 1, multiple);
}
}
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
namespace
{
size_t roundDownToMultiple(size_t num, size_t multiple)
{
return (num / multiple) * multiple;
}
size_t roundUpToMultiple(size_t num, size_t multiple)
{
return roundDownToMultiple(num + multiple - 1, multiple);
}
std::string getCommonUserID()
{
auto user_from_context = DB::Context::getGlobalContextInstance()->getFilesystemCacheUser();
const auto user = user_from_context.empty() ? toString(ServerUUID::get()) : user_from_context;
return user;
}
}
void FileCacheReserveStat::update(size_t size, FileSegmentKind kind, bool releasable) void FileCacheReserveStat::update(size_t size, FileSegmentKind kind, bool releasable)
{ {
auto & local_stat = stat_by_kind[kind]; auto & local_stat = stat_by_kind[kind];
@ -77,8 +84,9 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s
, bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0) , bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0)
, boundary_alignment(settings.boundary_alignment) , boundary_alignment(settings.boundary_alignment)
, load_metadata_threads(settings.load_metadata_threads) , load_metadata_threads(settings.load_metadata_threads)
, write_cache_per_user_directory(settings.write_cache_per_user_id_directory)
, log(&Poco::Logger::get("FileCache(" + cache_name + ")")) , log(&Poco::Logger::get("FileCache(" + cache_name + ")"))
, metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads) , metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads, write_cache_per_user_directory)
{ {
if (settings.cache_policy == "LRU") if (settings.cache_policy == "LRU")
main_priority = std::make_unique<LRUFileCachePriority>(settings.max_size, settings.max_elements); main_priority = std::make_unique<LRUFileCachePriority>(settings.max_size, settings.max_elements);
@ -101,19 +109,31 @@ FileCache::Key FileCache::createKeyForPath(const String & path)
return Key(path); return Key(path);
} }
const FileCache::UserInfo & FileCache::getCommonUser()
{
static UserInfo common_user(getCommonUserID(), 0);
return common_user;
}
const FileCache::UserInfo & FileCache::getInternalUser()
{
static UserInfo common_user("internal");
return common_user;
}
const String & FileCache::getBasePath() const const String & FileCache::getBasePath() const
{ {
return metadata.getBaseDirectory(); return metadata.getBaseDirectory();
} }
String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const String FileCache::getFileSegmentPath(const Key & key, size_t offset, FileSegmentKind segment_kind, const UserInfo & user) const
{ {
return metadata.getPathForFileSegment(key, offset, segment_kind); return metadata.getFileSegmentPath(key, offset, segment_kind, user);
} }
String FileCache::getPathInLocalCache(const Key & key) const String FileCache::getKeyPath(const Key & key, const UserInfo & user) const
{ {
return metadata.getPathForKey(key); return metadata.getKeyPath(key, user);
} }
void FileCache::assertInitialized() const void FileCache::assertInitialized() const
@ -276,9 +296,10 @@ std::vector<FileSegment::Range> FileCache::splitRange(size_t offset, size_t size
size_t remaining_size = size; size_t remaining_size = size;
FileSegments file_segments; FileSegments file_segments;
const size_t max_size = max_file_segment_size.load();
while (current_pos < end_pos_non_included) while (current_pos < end_pos_non_included)
{ {
auto current_file_segment_size = std::min(remaining_size, max_file_segment_size); auto current_file_segment_size = std::min(remaining_size, max_size);
ranges.emplace_back(current_pos, current_pos + current_file_segment_size - 1); ranges.emplace_back(current_pos, current_pos + current_file_segment_size - 1);
remaining_size -= current_file_segment_size; remaining_size -= current_file_segment_size;
@ -305,9 +326,10 @@ FileSegments FileCache::splitRangeIntoFileSegments(
size_t remaining_size = size; size_t remaining_size = size;
FileSegments file_segments; FileSegments file_segments;
const size_t max_size = max_file_segment_size.load();
while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit)) while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit))
{ {
current_file_segment_size = std::min(remaining_size, max_file_segment_size); current_file_segment_size = std::min(remaining_size, max_size);
remaining_size -= current_file_segment_size; remaining_size -= current_file_segment_size;
auto file_segment_metadata_it = addFileSegment( auto file_segment_metadata_it = addFileSegment(
@ -468,11 +490,12 @@ FileSegmentsHolderPtr FileCache::set(
const Key & key, const Key & key,
size_t offset, size_t offset,
size_t size, size_t size,
const CreateFileSegmentSettings & create_settings) const CreateFileSegmentSettings & create_settings,
const UserInfo & user)
{ {
assertInitialized(); assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user);
FileSegment::Range range(offset, offset + size - 1); FileSegment::Range range(offset, offset + size - 1);
auto file_segments = getImpl(*locked_key, range, /* file_segments_limit */0); auto file_segments = getImpl(*locked_key, range, /* file_segments_limit */0);
@ -502,7 +525,8 @@ FileCache::getOrSet(
size_t size, size_t size,
size_t file_size, size_t file_size,
const CreateFileSegmentSettings & create_settings, const CreateFileSegmentSettings & create_settings,
size_t file_segments_limit) size_t file_segments_limit,
const UserInfo & user)
{ {
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds); ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds);
@ -516,7 +540,7 @@ FileCache::getOrSet(
chassert(aligned_offset <= range.left); chassert(aligned_offset <= range.left);
chassert(aligned_end_offset >= range.right); chassert(aligned_end_offset >= range.right);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user);
/// Get all segments which intersect with the given range. /// Get all segments which intersect with the given range.
auto file_segments = getImpl(*locked_key, range, file_segments_limit); auto file_segments = getImpl(*locked_key, range, file_segments_limit);
@ -631,13 +655,18 @@ FileCache::getOrSet(
return std::make_unique<FileSegmentsHolder>(std::move(file_segments)); return std::make_unique<FileSegmentsHolder>(std::move(file_segments));
} }
FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size, size_t file_segments_limit) FileSegmentsHolderPtr FileCache::get(
const Key & key,
size_t offset,
size_t size,
size_t file_segments_limit,
const UserID & user_id)
{ {
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetMicroseconds); ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetMicroseconds);
assertInitialized(); assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL, UserInfo(user_id));
if (locked_key) if (locked_key)
{ {
FileSegment::Range range(offset, offset + size - 1); FileSegment::Range range(offset, offset + size - 1);
@ -705,7 +734,7 @@ KeyMetadata::iterator FileCache::addFileSegment(
auto & stash_records = stash->records; auto & stash_records = stash->records;
stash_records.emplace( stash_records.emplace(
stash_key, stash->queue->add(locked_key.getKeyMetadata(), offset, 0, *lock)); stash_key, stash->queue->add(locked_key.getKeyMetadata(), offset, 0, locked_key.getKeyMetadata()->user, *lock));
if (stash->queue->getElementsCount(*lock) > stash->queue->getElementsLimit(*lock)) if (stash->queue->getElementsCount(*lock) > stash->queue->getElementsLimit(*lock))
stash->queue->pop(*lock); stash->queue->pop(*lock);
@ -738,7 +767,11 @@ KeyMetadata::iterator FileCache::addFileSegment(
return file_segment_metadata_it; return file_segment_metadata_it;
} }
bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCacheReserveStat & reserve_stat) bool FileCache::tryReserve(
FileSegment & file_segment,
const size_t size,
FileCacheReserveStat & reserve_stat,
const UserInfo & user)
{ {
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheReserveMicroseconds); ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheReserveMicroseconds);
@ -780,7 +813,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
if (query_priority) if (query_priority)
{ {
if (!query_priority->collectCandidatesForEviction(size, reserve_stat, eviction_candidates, {}, finalize_eviction_func, cache_lock)) if (!query_priority->collectCandidatesForEviction(
size, reserve_stat, eviction_candidates, {}, finalize_eviction_func, user.user_id, cache_lock))
return false; return false;
LOG_TEST(log, "Query limits satisfied (while reserving for {}:{})", file_segment.key(), file_segment.offset()); LOG_TEST(log, "Query limits satisfied (while reserving for {}:{})", file_segment.key(), file_segment.offset());
@ -793,7 +827,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
auto queue_iterator = file_segment.getQueueIterator(); auto queue_iterator = file_segment.getQueueIterator();
chassert(!queue_iterator || file_segment.getReservedSize() > 0); chassert(!queue_iterator || file_segment.getReservedSize() > 0);
if (!main_priority->collectCandidatesForEviction(size, reserve_stat, eviction_candidates, queue_iterator, finalize_eviction_func, cache_lock)) if (!main_priority->collectCandidatesForEviction(
size, reserve_stat, eviction_candidates, queue_iterator, finalize_eviction_func, user.user_id, cache_lock))
return false; return false;
if (!file_segment.getKeyMetadata()->createBaseDirectory()) if (!file_segment.getKeyMetadata()->createBaseDirectory())
@ -812,7 +847,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
{ {
/// Space reservation is incremental, so file_segment_metadata is created first (with state empty), /// Space reservation is incremental, so file_segment_metadata is created first (with state empty),
/// and getQueueIterator() is assigned on first space reservation attempt. /// and getQueueIterator() is assigned on first space reservation attempt.
queue_iterator = main_priority->add(file_segment.getKeyMetadata(), file_segment.offset(), size, cache_lock); queue_iterator = main_priority->add(file_segment.getKeyMetadata(), file_segment.offset(), size, user, cache_lock);
file_segment.setQueueIterator(queue_iterator); file_segment.setQueueIterator(queue_iterator);
} }
@ -825,7 +860,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
if (query_queue_it) if (query_queue_it)
query_queue_it->updateSize(size); query_queue_it->updateSize(size);
else else
query_context->add(file_segment.getKeyMetadata(), file_segment.offset(), size, cache_lock); query_context->add(file_segment.getKeyMetadata(), file_segment.offset(), size, user, cache_lock);
} }
if (main_priority->getSize(cache_lock) > (1ull << 63)) if (main_priority->getSize(cache_lock) > (1ull << 63))
@ -834,40 +869,40 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
return true; return true;
} }
void FileCache::iterate(IterateFunc && func) void FileCache::iterate(IterateFunc && func, const UserID & user_id)
{ {
return metadata.iterate([&](const LockedKey & locked_key) return metadata.iterate([&](const LockedKey & locked_key)
{ {
for (const auto & file_segment_metadata : locked_key) for (const auto & file_segment_metadata : locked_key)
func(FileSegment::getInfo(file_segment_metadata.second->file_segment)); func(FileSegment::getInfo(file_segment_metadata.second->file_segment));
}); }, user_id);
} }
void FileCache::removeKey(const Key & key) void FileCache::removeKey(const Key & key, const UserID & user_id)
{ {
assertInitialized(); assertInitialized();
metadata.removeKey(key, /* if_exists */false, /* if_releasable */true); metadata.removeKey(key, /* if_exists */false, /* if_releasable */true, user_id);
} }
void FileCache::removeKeyIfExists(const Key & key) void FileCache::removeKeyIfExists(const Key & key, const UserID & user_id)
{ {
assertInitialized(); assertInitialized();
metadata.removeKey(key, /* if_exists */true, /* if_releasable */true); metadata.removeKey(key, /* if_exists */true, /* if_releasable */true, user_id);
} }
void FileCache::removeFileSegment(const Key & key, size_t offset) void FileCache::removeFileSegment(const Key & key, size_t offset, const UserID & user_id)
{ {
assertInitialized(); assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW, UserInfo(user_id));
locked_key->removeFileSegment(offset); locked_key->removeFileSegment(offset);
} }
void FileCache::removePathIfExists(const String & path) void FileCache::removePathIfExists(const String & path, const UserID & user_id)
{ {
removeKeyIfExists(createKeyForPath(path)); removeKeyIfExists(createKeyForPath(path), user_id);
} }
void FileCache::removeAllReleasable() void FileCache::removeAllReleasable(const UserID & user_id)
{ {
assertInitialized(); assertInitialized();
@ -875,7 +910,7 @@ void FileCache::removeAllReleasable()
assertCacheCorrectness(); assertCacheCorrectness();
#endif #endif
metadata.removeAllKeys(/* if_releasable */true); metadata.removeAllKeys(/* if_releasable */true, user_id);
if (stash) if (stash)
{ {
@ -1002,6 +1037,24 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
return; return;
} }
UserInfo user;
if (write_cache_per_user_directory)
{
auto filename = keys_dir.filename().string();
auto pos = filename.find_last_of('.');
if (pos == std::string::npos)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected file format: {}", filename);
user = UserInfo(filename.substr(0, pos), parse<UInt64>(filename.substr(pos + 1)));
LOG_TEST(log, "Loading cache for user {}", user.user_id);
}
else
{
user = getCommonUser();
}
UInt64 offset = 0, size = 0; UInt64 offset = 0, size = 0;
for (; key_it != fs::directory_iterator(); key_it++) for (; key_it != fs::directory_iterator(); key_it++)
{ {
@ -1024,7 +1077,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
} }
const auto key = Key::fromKeyString(key_directory.filename().string()); const auto key = Key::fromKeyString(key_directory.filename().string());
auto key_metadata = metadata.getKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true); auto key_metadata = metadata.getKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user, /* is_initial_load */true);
for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it) for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it)
{ {
@ -1072,9 +1125,9 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
auto lock = lockCache(); auto lock = lockCache();
size_limit = main_priority->getSizeLimit(lock); size_limit = main_priority->getSizeLimit(lock);
limits_satisfied = main_priority->canFit(size, lock); limits_satisfied = main_priority->canFit(size, lock, nullptr, true);
if (limits_satisfied) if (limits_satisfied)
cache_it = main_priority->add(key_metadata, offset, size, lock, /* is_startup */true); cache_it = main_priority->add(key_metadata, offset, size, user, lock, /* best_effort */true);
/// TODO: we can get rid of this lockCache() if we first load everything in parallel /// TODO: we can get rid of this lockCache() if we first load everything in parallel
/// without any mutual lock between loading threads, and only after do removeOverflow(). /// without any mutual lock between loading threads, and only after do removeOverflow().
@ -1128,7 +1181,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
if (key_metadata->sizeUnlocked() == 0) if (key_metadata->sizeUnlocked() == 0)
{ {
metadata.removeKey(key, false, false); metadata.removeKey(key, false, false, getInternalUser().user_id);
} }
} }
} }
@ -1147,7 +1200,7 @@ void FileCache::deactivateBackgroundOperations()
metadata.shutdown(); metadata.shutdown();
} }
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos() std::vector<FileSegment::Info> FileCache::getFileSegmentInfos(const UserID & user_id)
{ {
assertInitialized(); assertInitialized();
#ifndef NDEBUG #ifndef NDEBUG
@ -1159,20 +1212,20 @@ std::vector<FileSegment::Info> FileCache::getFileSegmentInfos()
{ {
for (const auto & [_, file_segment_metadata] : locked_key) for (const auto & [_, file_segment_metadata] : locked_key)
file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment)); file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment));
}); }, user_id);
return file_segments; return file_segments;
} }
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos(const Key & key) std::vector<FileSegment::Info> FileCache::getFileSegmentInfos(const Key & key, const UserID & user_id)
{ {
std::vector<FileSegment::Info> file_segments; std::vector<FileSegment::Info> file_segments;
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL, UserInfo(user_id));
for (const auto & [_, file_segment_metadata] : *locked_key) for (const auto & [_, file_segment_metadata] : *locked_key)
file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment)); file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment));
return file_segments; return file_segments;
} }
std::vector<FileSegment::Info> FileCache::dumpQueue() IFileCachePriority::PriorityDumpPtr FileCache::dumpQueue()
{ {
assertInitialized(); assertInitialized();
return main_priority->dump(lockCache()); return main_priority->dump(lockCache());
@ -1182,7 +1235,7 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
{ {
assertInitialized(); assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL, getInternalUser());
if (!locked_key) if (!locked_key)
return {}; return {};
@ -1190,8 +1243,9 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
for (const auto & [offset, file_segment_metadata] : *locked_key) for (const auto & [offset, file_segment_metadata] : *locked_key)
{ {
if (file_segment_metadata->file_segment->state() == FileSegment::State::DOWNLOADED) const auto & file_segment = *file_segment_metadata->file_segment;
cache_paths.push_back(metadata.getPathForFileSegment(key, offset, file_segment_metadata->file_segment->getKind())); if (file_segment.state() == FileSegment::State::DOWNLOADED)
cache_paths.push_back(locked_key->getKeyMetadata()->getFileSegmentPath(file_segment));
} }
return cache_paths; return cache_paths;
} }
@ -1214,7 +1268,7 @@ void FileCache::assertCacheCorrectness()
{ {
chassert(file_segment_metadata->file_segment->assertCorrectness()); chassert(file_segment_metadata->file_segment->assertCorrectness());
} }
}); }, getInternalUser().user_id);
} }
void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings) void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings)
@ -1286,26 +1340,10 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
actual_settings.max_elements = main_priority->getElementsLimit(cache_lock); actual_settings.max_elements = main_priority->getElementsLimit(cache_lock);
} }
} }
}
FileCache::QueryContextHolder::QueryContextHolder( if (new_settings.max_file_segment_size != actual_settings.max_file_segment_size)
const String & query_id_,
FileCache * cache_,
FileCacheQueryLimit::QueryContextPtr context_)
: query_id(query_id_)
, cache(cache_)
, context(context_)
{
}
FileCache::QueryContextHolder::~QueryContextHolder()
{
/// If only the query_map and the current holder hold the context_query,
/// the query has been completed and the query_context is released.
if (context && context.use_count() == 2)
{ {
auto lock = cache->lockCache(); max_file_segment_size = actual_settings.max_file_segment_size = new_settings.max_file_segment_size;
cache->query_limit->removeQueryContext(query_id, lock);
} }
} }
@ -1317,7 +1355,7 @@ FileCache::QueryContextHolderPtr FileCache::getQueryContextHolder(
auto lock = lockCache(); auto lock = lockCache();
auto context = query_limit->getOrSetQueryContext(query_id, read_settings, lock); auto context = query_limit->getOrSetQueryContext(query_id, read_settings, lock);
return std::make_unique<QueryContextHolder>(query_id, this, std::move(context)); return std::make_unique<QueryContextHolder>(query_id, this, query_limit.get(), std::move(context));
} }
std::vector<FileSegment::Info> FileCache::sync() std::vector<FileSegment::Info> FileCache::sync()
@ -1327,7 +1365,7 @@ std::vector<FileSegment::Info> FileCache::sync()
{ {
auto broken = locked_key.sync(); auto broken = locked_key.sync();
file_segments.insert(file_segments.end(), broken.begin(), broken.end()); file_segments.insert(file_segments.end(), broken.begin(), broken.end());
}); }, getInternalUser().user_id);
return file_segments; return file_segments;
} }

View File

@ -17,6 +17,7 @@
#include <Interpreters/Cache/QueryLimit.h> #include <Interpreters/Cache/QueryLimit.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h> #include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <Interpreters/Cache/FileCacheSettings.h> #include <Interpreters/Cache/FileCacheSettings.h>
#include <Interpreters/Cache/UserInfo.h>
#include <filesystem> #include <filesystem>
@ -34,12 +35,29 @@ struct FileCacheReserveStat
size_t non_releasable_size = 0; size_t non_releasable_size = 0;
size_t non_releasable_count = 0; size_t non_releasable_count = 0;
Stat & operator +=(const Stat & other)
{
releasable_size += other.releasable_size;
releasable_count += other.releasable_count;
non_releasable_size += other.non_releasable_size;
non_releasable_count += other.non_releasable_count;
return *this;
}
}; };
Stat stat; Stat stat;
std::unordered_map<FileSegmentKind, Stat> stat_by_kind; std::unordered_map<FileSegmentKind, Stat> stat_by_kind;
void update(size_t size, FileSegmentKind kind, bool releasable); void update(size_t size, FileSegmentKind kind, bool releasable);
FileCacheReserveStat & operator +=(const FileCacheReserveStat & other)
{
stat += other.stat;
for (const auto & [name, stat_] : other.stat_by_kind)
stat_by_kind[name] += stat_;
return *this;
}
}; };
/// Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments. /// Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments.
@ -51,6 +69,9 @@ public:
using QueryLimit = DB::FileCacheQueryLimit; using QueryLimit = DB::FileCacheQueryLimit;
using Priority = IFileCachePriority; using Priority = IFileCachePriority;
using PriorityEntry = IFileCachePriority::Entry; using PriorityEntry = IFileCachePriority::Entry;
using QueryContextHolder = FileCacheQueryLimit::QueryContextHolder;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
FileCache(const std::string & cache_name, const FileCacheSettings & settings); FileCache(const std::string & cache_name, const FileCacheSettings & settings);
@ -62,9 +83,13 @@ public:
static Key createKeyForPath(const String & path); static Key createKeyForPath(const String & path);
String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const; static const UserInfo & getCommonUser();
String getPathInLocalCache(const Key & key) const; static const UserInfo & getInternalUser();
String getFileSegmentPath(const Key & key, size_t offset, FileSegmentKind segment_kind, const UserInfo & user) const;
String getKeyPath(const Key & key, const UserInfo & user) const;
/** /**
* Given an `offset` and `size` representing [offset, offset + size) bytes interval, * Given an `offset` and `size` representing [offset, offset + size) bytes interval,
@ -83,7 +108,8 @@ public:
size_t size, size_t size,
size_t file_size, size_t file_size,
const CreateFileSegmentSettings & settings, const CreateFileSegmentSettings & settings,
size_t file_segments_limit = 0); size_t file_segments_limit,
const UserInfo & user);
/** /**
* Segments in returned list are ordered in ascending order and represent a full contiguous * Segments in returned list are ordered in ascending order and represent a full contiguous
@ -94,24 +120,34 @@ public:
* with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change * with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change
* it's state (and become DOWNLOADED). * it's state (and become DOWNLOADED).
*/ */
FileSegmentsHolderPtr get(const Key & key, size_t offset, size_t size, size_t file_segments_limit); FileSegmentsHolderPtr get(
const Key & key,
size_t offset,
size_t size,
size_t file_segments_limit,
const UserID & user_id);
FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); FileSegmentsHolderPtr set(
const Key & key,
size_t offset,
size_t size,
const CreateFileSegmentSettings & settings,
const UserInfo & user);
/// Remove file segment by `key` and `offset`. Throws if file segment does not exist. /// Remove file segment by `key` and `offset`. Throws if file segment does not exist.
void removeFileSegment(const Key & key, size_t offset); void removeFileSegment(const Key & key, size_t offset, const UserID & user_id);
/// Remove files by `key`. Throws if key does not exist. /// Remove files by `key`. Throws if key does not exist.
void removeKey(const Key & key); void removeKey(const Key & key, const UserID & user_id);
/// Remove files by `key`. /// Remove files by `key`.
void removeKeyIfExists(const Key & key); void removeKeyIfExists(const Key & key, const UserID & user_id);
/// Removes files by `path`. /// Removes files by `path`.
void removePathIfExists(const String & path); void removePathIfExists(const String & path, const UserID & user_id);
/// Remove files by `key`. /// Remove files by `key`.
void removeAllReleasable(); void removeAllReleasable(const UserID & user_id);
std::vector<String> tryGetCachePaths(const Key & key); std::vector<String> tryGetCachePaths(const Key & key);
@ -121,48 +157,41 @@ public:
size_t getMaxFileSegmentSize() const { return max_file_segment_size; } size_t getMaxFileSegmentSize() const { return max_file_segment_size; }
bool tryReserve(FileSegment & file_segment, size_t size, FileCacheReserveStat & stat); bool tryReserve(
FileSegment & file_segment,
size_t size,
FileCacheReserveStat & stat,
const UserInfo & user);
std::vector<FileSegment::Info> getFileSegmentInfos(); std::vector<FileSegment::Info> getFileSegmentInfos(const UserID & user_id);
std::vector<FileSegment::Info> getFileSegmentInfos(const Key & key); std::vector<FileSegment::Info> getFileSegmentInfos(const Key & key, const UserID & user_id);
std::vector<FileSegment::Info> dumpQueue();
IFileCachePriority::PriorityDumpPtr dumpQueue();
void deactivateBackgroundOperations(); void deactivateBackgroundOperations();
/// For per query cache limit.
struct QueryContextHolder : private boost::noncopyable
{
QueryContextHolder(const String & query_id_, FileCache * cache_, QueryLimit::QueryContextPtr context_);
QueryContextHolder() = default;
~QueryContextHolder();
String query_id;
FileCache * cache = nullptr;
QueryLimit::QueryContextPtr context;
};
using QueryContextHolderPtr = std::unique_ptr<QueryContextHolder>;
QueryContextHolderPtr getQueryContextHolder(const String & query_id, const ReadSettings & settings);
CacheGuard::Lock lockCache() const; CacheGuard::Lock lockCache() const;
std::vector<FileSegment::Info> sync(); std::vector<FileSegment::Info> sync();
using QueryContextHolderPtr = std::unique_ptr<QueryContextHolder>;
QueryContextHolderPtr getQueryContextHolder(const String & query_id, const ReadSettings & settings);
using IterateFunc = std::function<void(const FileSegmentInfo &)>; using IterateFunc = std::function<void(const FileSegmentInfo &)>;
void iterate(IterateFunc && func); void iterate(IterateFunc && func, const UserID & user_id);
void applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings); void applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings);
private: private:
using KeyAndOffset = FileCacheKeyAndOffset; using KeyAndOffset = FileCacheKeyAndOffset;
const size_t max_file_segment_size; std::atomic<size_t> max_file_segment_size;
const size_t bypass_cache_threshold; const size_t bypass_cache_threshold;
const size_t boundary_alignment; const size_t boundary_alignment;
size_t load_metadata_threads; size_t load_metadata_threads;
const bool write_cache_per_user_directory;
Poco::Logger * log; Poco::Logger * log;

View File

@ -21,6 +21,7 @@ struct FileCacheKey
static FileCacheKey random(); static FileCacheKey random();
bool operator==(const FileCacheKey & other) const { return key == other.key; } bool operator==(const FileCacheKey & other) const { return key == other.key; }
bool operator<(const FileCacheKey & other) const { return key < other.key; }
static FileCacheKey fromKeyString(const std::string & key_str); static FileCacheKey fromKeyString(const std::string & key_str);
}; };

View File

@ -5,6 +5,7 @@
#include <Common/NamedCollections/NamedCollections.h> #include <Common/NamedCollections/NamedCollections.h>
#include <boost/algorithm/string/case_conv.hpp> #include <boost/algorithm/string/case_conv.hpp>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <Common/logger_useful.h>
namespace DB namespace DB
{ {
@ -74,6 +75,9 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin
if (has("slru_size_ratio")) if (has("slru_size_ratio"))
slru_size_ratio = get_double("slru_size_ratio"); slru_size_ratio = get_double("slru_size_ratio");
if (has("write_cache_per_user_id_directory"))
slru_size_ratio = get_uint("write_cache_per_user_id_directory");
} }
void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)

View File

@ -33,6 +33,8 @@ struct FileCacheSettings
size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS; size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS;
bool write_cache_per_user_id_directory = false;
std::string cache_policy = "LRU"; std::string cache_policy = "LRU";
double slru_size_ratio = 0.5; double slru_size_ratio = 0.5;

View File

@ -85,7 +85,7 @@ FileSegment::FileSegment(
case (State::DOWNLOADED): case (State::DOWNLOADED):
{ {
reserved_size = downloaded_size = size_; reserved_size = downloaded_size = size_;
chassert(fs::file_size(getPathInLocalCache()) == size_); chassert(fs::file_size(getPath()) == size_);
chassert(queue_iterator); chassert(queue_iterator);
chassert(key_metadata.lock()); chassert(key_metadata.lock());
break; break;
@ -115,12 +115,12 @@ FileSegment::State FileSegment::state() const
return download_state; return download_state;
} }
String FileSegment::getPathInLocalCache() const String FileSegment::getPath() const
{ {
return getKeyMetadata()->getFileSegmentPath(*this); return getKeyMetadata()->getFileSegmentPath(*this);
} }
String FileSegment::tryGetPathInLocalCache() const String FileSegment::tryGetPath() const
{ {
auto metadata = tryGetKeyMetadata(); auto metadata = tryGetKeyMetadata();
if (!metadata) if (!metadata)
@ -182,7 +182,7 @@ void FileSegment::setDownloadedSize(size_t delta)
{ {
auto lock = lockFileSegment(); auto lock = lockFileSegment();
downloaded_size += delta; downloaded_size += delta;
assert(downloaded_size == std::filesystem::file_size(getPathInLocalCache())); assert(downloaded_size == std::filesystem::file_size(getPath()));
} }
bool FileSegment::isDownloaded() const bool FileSegment::isDownloaded() const
@ -339,7 +339,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
if (!size) if (!size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed");
const auto file_segment_path = getPathInLocalCache(); const auto file_segment_path = getPath();
{ {
auto lock = lockFileSegment(); auto lock = lockFileSegment();
@ -531,7 +531,7 @@ bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve
if (!reserve_stat) if (!reserve_stat)
reserve_stat = &dummy_stat; reserve_stat = &dummy_stat;
bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat); bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat, getKeyMetadata()->user);
if (!reserved) if (!reserved)
setDownloadFailedUnlocked(lockFileSegment()); setDownloadFailedUnlocked(lockFileSegment());
@ -554,7 +554,7 @@ void FileSegment::setDownloadedUnlocked(const FileSegmentGuard::Lock &)
} }
chassert(downloaded_size > 0); chassert(downloaded_size > 0);
chassert(fs::file_size(getPathInLocalCache()) == downloaded_size); chassert(fs::file_size(getPath()) == downloaded_size);
} }
void FileSegment::setDownloadFailed() void FileSegment::setDownloadFailed()
@ -654,7 +654,7 @@ void FileSegment::complete()
case State::DOWNLOADED: case State::DOWNLOADED:
{ {
chassert(current_downloaded_size == range().size()); chassert(current_downloaded_size == range().size());
chassert(current_downloaded_size == fs::file_size(getPathInLocalCache())); chassert(current_downloaded_size == fs::file_size(getPath()));
chassert(!cache_writer); chassert(!cache_writer);
chassert(!remote_file_reader); chassert(!remote_file_reader);
break; break;
@ -800,7 +800,7 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons
chassert(downloaded_size == reserved_size); chassert(downloaded_size == reserved_size);
chassert(downloaded_size == range().size()); chassert(downloaded_size == range().size());
chassert(downloaded_size > 0); chassert(downloaded_size > 0);
chassert(std::filesystem::file_size(getPathInLocalCache()) > 0); chassert(std::filesystem::file_size(getPath()) > 0);
chassert(queue_iterator); chassert(queue_iterator);
check_iterator(queue_iterator); check_iterator(queue_iterator);
} }
@ -844,10 +844,11 @@ void FileSegment::assertNotDetachedUnlocked(const FileSegmentGuard::Lock & lock)
FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment) FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment)
{ {
auto lock = file_segment->lockFileSegment(); auto lock = file_segment->lockFileSegment();
auto key_metadata = file_segment->tryGetKeyMetadata();
return Info{ return Info{
.key = file_segment->key(), .key = file_segment->key(),
.offset = file_segment->offset(), .offset = file_segment->offset(),
.path = file_segment->tryGetPathInLocalCache(), .path = file_segment->tryGetPath(),
.range_left = file_segment->range().left, .range_left = file_segment->range().left,
.range_right = file_segment->range().right, .range_right = file_segment->range().right,
.kind = file_segment->segment_kind, .kind = file_segment->segment_kind,
@ -858,6 +859,8 @@ FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment)
.references = static_cast<uint64_t>(file_segment.use_count()), .references = static_cast<uint64_t>(file_segment.use_count()),
.is_unbound = file_segment->is_unbound, .is_unbound = file_segment->is_unbound,
.queue_entry_type = file_segment->queue_iterator ? file_segment->queue_iterator->getType() : QueueEntryType::None, .queue_entry_type = file_segment->queue_iterator ? file_segment->queue_iterator->getType() : QueueEntryType::None,
.user_id = key_metadata->user.user_id,
.user_weight = key_metadata->user.weight.value(),
}; };
} }
@ -890,7 +893,6 @@ void FileSegment::setDetachedState(const FileSegmentGuard::Lock & lock)
{ {
setDownloadState(State::DETACHED, lock); setDownloadState(State::DETACHED, lock);
key_metadata.reset(); key_metadata.reset();
cache = nullptr;
queue_iterator = nullptr; queue_iterator = nullptr;
try try
{ {

View File

@ -112,7 +112,7 @@ public:
bool isUnbound() const { return is_unbound; } bool isUnbound() const { return is_unbound; }
String getPathInLocalCache() const; String getPath() const;
int getFlagsForLocalRead() const { return O_RDONLY | O_CLOEXEC; } int getFlagsForLocalRead() const { return O_RDONLY | O_CLOEXEC; }
@ -243,7 +243,7 @@ private:
LockedKeyPtr lockKeyMetadata(bool assert_exists = true) const; LockedKeyPtr lockKeyMetadata(bool assert_exists = true) const;
FileSegmentGuard::Lock lockFileSegment() const; FileSegmentGuard::Lock lockFileSegment() const;
String tryGetPathInLocalCache() const; String tryGetPath() const;
Key file_key; Key file_key;
Range segment_range; Range segment_range;

View File

@ -78,5 +78,7 @@ namespace DB
uint64_t references; uint64_t references;
bool is_unbound; bool is_unbound;
FileCacheQueueEntryType queue_entry_type; FileCacheQueueEntryType queue_entry_type;
std::string user_id;
uint64_t user_weight;
}; };
} }

View File

@ -7,6 +7,7 @@
#include <Interpreters/Cache/Guards.h> #include <Interpreters/Cache/Guards.h>
#include <Interpreters/Cache/IFileCachePriority.h> #include <Interpreters/Cache/IFileCachePriority.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h> #include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <Interpreters/Cache/UserInfo.h>
namespace DB namespace DB
{ {
@ -18,6 +19,8 @@ class IFileCachePriority : private boost::noncopyable
public: public:
using Key = FileCacheKey; using Key = FileCacheKey;
using QueueEntryType = FileCacheQueueEntryType; using QueueEntryType = FileCacheQueueEntryType;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
struct Entry struct Entry
{ {
@ -52,8 +55,6 @@ public:
}; };
using IteratorPtr = std::shared_ptr<Iterator>; using IteratorPtr = std::shared_ptr<Iterator>;
IFileCachePriority(size_t max_size_, size_t max_elements_);
virtual ~IFileCachePriority() = default; virtual ~IFileCachePriority() = default;
size_t getElementsLimit(const CacheGuard::Lock &) const { return max_elements; } size_t getElementsLimit(const CacheGuard::Lock &) const { return max_elements; }
@ -69,14 +70,28 @@ public:
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const UserInfo & user,
const CacheGuard::Lock &, const CacheGuard::Lock &,
bool is_startup = false) = 0; bool best_effort = false) = 0;
virtual bool canFit(size_t size, const CacheGuard::Lock &) const = 0; /// `reservee` is the entry for which are reserving now.
/// It does not exist, if it is the first space reservation attempt
/// for the corresponding file segment.
virtual bool canFit( /// NOLINT
size_t size,
const CacheGuard::Lock &,
IteratorPtr reservee = nullptr,
bool best_effort = false) const = 0;
virtual void shuffle(const CacheGuard::Lock &) = 0; virtual void shuffle(const CacheGuard::Lock &) = 0;
virtual std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) = 0; struct IPriorityDump
{
virtual ~IPriorityDump() = default;
};
using PriorityDumpPtr = std::shared_ptr<IPriorityDump>;
virtual PriorityDumpPtr 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(
@ -85,11 +100,14 @@ public:
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock &) = 0; const CacheGuard::Lock &) = 0;
virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) = 0; virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) = 0;
protected: protected:
IFileCachePriority(size_t max_size_, size_t max_elements_);
size_t max_size = 0; size_t max_size = 0;
size_t max_elements = 0; size_t max_elements = 0;
}; };

View File

@ -29,12 +29,22 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
LRUFileCachePriority::LRUFileCachePriority(size_t max_size_, size_t max_elements_, StatePtr state_)
: IFileCachePriority(max_size_, max_elements_)
{
if (state_)
state = state_;
else
state = std::make_shared<State>();
}
IFileCachePriority::IteratorPtr LRUFileCachePriority::add( /// NOLINT IFileCachePriority::IteratorPtr LRUFileCachePriority::add( /// NOLINT
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const UserInfo &,
const CacheGuard::Lock & lock, const CacheGuard::Lock & lock,
bool /* is_startup */) bool)
{ {
return std::make_shared<LRUIterator>(add(std::make_shared<Entry>(key_metadata->key, offset, size, key_metadata), lock)); return std::make_shared<LRUIterator>(add(std::make_shared<Entry>(key_metadata->key, offset, size, key_metadata), lock));
} }
@ -63,12 +73,12 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(EntryPtr entry, cons
#endif #endif
const auto & size_limit = getSizeLimit(lock); const auto & size_limit = getSizeLimit(lock);
if (size_limit && current_size + entry->size > size_limit) if (size_limit && state->current_size + entry->size > size_limit)
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Not enough space to add {}:{} with size {}: current size: {}/{}", "Not enough space to add {}:{} with size {}: current size: {}/{}",
entry->key, entry->offset, entry->size, current_size, size_limit); entry->key, entry->offset, entry->size, state->current_size, size_limit);
} }
auto iterator = queue.insert(queue.end(), entry); auto iterator = queue.insert(queue.end(), entry);
@ -102,13 +112,13 @@ LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::
void LRUFileCachePriority::updateSize(int64_t size) void LRUFileCachePriority::updateSize(int64_t size)
{ {
current_size += size; state->current_size += size;
CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size);
} }
void LRUFileCachePriority::updateElementsCount(int64_t num) void LRUFileCachePriority::updateElementsCount(int64_t num)
{ {
current_elements_num += num; state->current_elements_num += num;
CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements, num); CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements, num);
} }
@ -189,7 +199,11 @@ void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock &
} }
} }
bool LRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) const bool LRUFileCachePriority::canFit( /// NOLINT
size_t size,
const CacheGuard::Lock & lock,
IteratorPtr,
bool) const
{ {
return canFit(size, 0, 0, lock); return canFit(size, 0, 0, lock);
} }
@ -200,8 +214,8 @@ bool LRUFileCachePriority::canFit(
size_t released_elements_assumption, size_t released_elements_assumption,
const CacheGuard::Lock &) const const CacheGuard::Lock &) const
{ {
return (max_size == 0 || (current_size + size - released_size_assumption <= max_size)) return (max_size == 0 || (state->current_size + size - released_size_assumption <= max_size))
&& (max_elements == 0 || current_elements_num + 1 - released_elements_assumption <= max_elements); && (max_elements == 0 || state->current_elements_num + 1 - released_elements_assumption <= max_elements);
} }
bool LRUFileCachePriority::collectCandidatesForEviction( bool LRUFileCachePriority::collectCandidatesForEviction(
@ -210,6 +224,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::IteratorPtr, IFileCachePriority::IteratorPtr,
FinalizeEvictionFunc &, FinalizeEvictionFunc &,
const UserID &,
const CacheGuard::Lock & lock) const CacheGuard::Lock & lock)
{ {
if (canFit(size, lock)) if (canFit(size, lock))
@ -282,7 +297,7 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, L
return LRUIterator(this, it.iterator); return LRUIterator(this, it.iterator);
} }
std::vector<FileSegmentInfo> LRUFileCachePriority::dump(const CacheGuard::Lock & lock) IFileCachePriority::PriorityDumpPtr LRUFileCachePriority::dump(const CacheGuard::Lock & lock)
{ {
std::vector<FileSegmentInfo> res; std::vector<FileSegmentInfo> res;
iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata) iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata)
@ -290,7 +305,7 @@ std::vector<FileSegmentInfo> LRUFileCachePriority::dump(const CacheGuard::Lock &
res.emplace_back(FileSegment::getInfo(segment_metadata->file_segment)); res.emplace_back(FileSegment::getInfo(segment_metadata->file_segment));
return IterationResult::CONTINUE; return IterationResult::CONTINUE;
}, lock); }, lock);
return res; return std::make_shared<LRUPriorityDump>(res);
} }
bool LRUFileCachePriority::modifySizeLimits( bool LRUFileCachePriority::modifySizeLimits(
@ -301,8 +316,8 @@ bool LRUFileCachePriority::modifySizeLimits(
auto check_limits_satisfied = [&]() auto check_limits_satisfied = [&]()
{ {
return (max_size_ == 0 || current_size <= max_size_) return (max_size_ == 0 || state->current_size <= max_size_)
&& (max_elements_ == 0 || current_elements_num <= max_elements_); && (max_elements_ == 0 || state->current_elements_num <= max_elements_);
}; };
if (check_limits_satisfied()) if (check_limits_satisfied())

View File

@ -13,26 +13,35 @@ namespace DB
/// 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 final : public IFileCachePriority class LRUFileCachePriority final : public IFileCachePriority
{ {
private: friend class OvercommitFileCachePriority;
class LRUIterator; protected:
using LRUQueue = std::list<EntryPtr>; struct State
friend class SLRUFileCachePriority; {
std::atomic<size_t> current_size = 0;
std::atomic<size_t> current_elements_num = 0;
};
using StatePtr = std::shared_ptr<State>;
public: public:
LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) {} LRUFileCachePriority(size_t max_size_, size_t max_elements_, StatePtr state_ = nullptr);
size_t getSize(const CacheGuard::Lock &) const override { return current_size; } size_t getSize(const CacheGuard::Lock &) const override { return state->current_size; }
size_t getElementsCount(const CacheGuard::Lock &) const override { return current_elements_num; } size_t getElementsCount(const CacheGuard::Lock &) const override { return state->current_elements_num; }
bool canFit(size_t size, const CacheGuard::Lock &) const override; bool canFit( /// NOLINT
size_t size,
const CacheGuard::Lock &,
IteratorPtr reservee = nullptr,
bool best_effort = false) const override;
IteratorPtr add( /// NOLINT IteratorPtr add( /// NOLINT
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const UserInfo & user,
const CacheGuard::Lock &, const CacheGuard::Lock &,
bool is_startup = false) override; bool best_effort = false) override;
bool collectCandidatesForEviction( bool collectCandidatesForEviction(
size_t size, size_t size,
@ -40,27 +49,34 @@ public:
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock &) override; const CacheGuard::Lock &) override;
void shuffle(const CacheGuard::Lock &) override; void shuffle(const CacheGuard::Lock &) override;
std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) override; struct LRUPriorityDump : public IPriorityDump
{
std::vector<FileSegmentInfo> infos;
explicit LRUPriorityDump(const std::vector<FileSegmentInfo> & infos_) : infos(infos_) {}
void merge(const LRUPriorityDump & other) { infos.insert(infos.end(), other.infos.begin(), other.infos.end()); }
};
PriorityDumpPtr dump(const CacheGuard::Lock &) override;
void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); } void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); }
bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override; bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override;
private: private:
void updateElementsCount(int64_t num); class LRUIterator;
void updateSize(int64_t size); using LRUQueue = std::list<EntryPtr>;
friend class SLRUFileCachePriority;
LRUQueue queue; LRUQueue queue;
Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority"); Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority");
StatePtr state;
std::atomic<size_t> current_size = 0; void updateElementsCount(int64_t num);
/// current_elements_num is not always equal to queue.size() void updateSize(int64_t size);
/// because of invalidated entries.
std::atomic<size_t> current_elements_num = 0;
bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const; bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const;

View File

@ -26,6 +26,7 @@ namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int FILECACHE_ACCESS_DENIED;
} }
FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_) FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_)
@ -58,22 +59,36 @@ size_t FileSegmentMetadata::size() const
KeyMetadata::KeyMetadata( KeyMetadata::KeyMetadata(
const Key & key_, const Key & key_,
const std::string & key_path_, const UserInfo & user_,
CleanupQueuePtr cleanup_queue_, const CacheMetadata * cache_metadata_,
DownloadQueuePtr download_queue_,
Poco::Logger * log_,
std::shared_mutex & key_prefix_directory_mutex_,
bool created_base_directory_) bool created_base_directory_)
: key(key_) : key(key_)
, key_path(key_path_) , user(user_)
, cleanup_queue(cleanup_queue_) , cache_metadata(cache_metadata_)
, download_queue(download_queue_)
, key_prefix_directory_mutex(key_prefix_directory_mutex_)
, created_base_directory(created_base_directory_) , created_base_directory(created_base_directory_)
, log(log_)
{ {
if (created_base_directory) if (user_ == FileCache::getInternalUser())
chassert(fs::exists(key_path)); throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key metadata with internal user id");
if (!user_.weight.has_value())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key metadata withouot user weight");
chassert(!created_base_directory || fs::exists(getPath()));
}
bool KeyMetadata::checkAccess(const UserID & user_id_) const
{
return user_id_ == user.user_id || user_id_ == FileCache::getInternalUser().user_id;
}
void KeyMetadata::assertAccess(const UserID & user_id_) const
{
if (!checkAccess(user_id_))
{
throw Exception(ErrorCodes::FILECACHE_ACCESS_DENIED,
"Metadata for key {} belongs to user {}, but user {} requested it",
key.toString(), user.user_id, user_id_);
}
} }
LockedKeyPtr KeyMetadata::lock() LockedKeyPtr KeyMetadata::lock()
@ -108,8 +123,8 @@ bool KeyMetadata::createBaseDirectory()
{ {
try try
{ {
std::shared_lock lock(key_prefix_directory_mutex); std::shared_lock lock(cache_metadata->key_prefix_directory_mutex);
fs::create_directories(key_path); fs::create_directories(getPath());
} }
catch (const fs::filesystem_error & e) catch (const fs::filesystem_error & e)
{ {
@ -117,7 +132,7 @@ bool KeyMetadata::createBaseDirectory()
if (e.code() == std::errc::no_space_on_device) if (e.code() == std::errc::no_space_on_device)
{ {
LOG_TRACE(log, "Failed to create base directory for key {}, " LOG_TRACE(cache_metadata->log, "Failed to create base directory for key {}, "
"because no space left on device", key); "because no space left on device", key);
return false; return false;
@ -128,16 +143,30 @@ bool KeyMetadata::createBaseDirectory()
return true; return true;
} }
std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) const std::string KeyMetadata::getPath() const
{ {
return fs::path(key_path) return cache_metadata->getKeyPath(key, user);
/ CacheMetadata::getFileNameForFileSegment(file_segment.offset(), file_segment.getKind());
} }
CacheMetadata::CacheMetadata(const std::string & path_, size_t background_download_queue_size_limit_, size_t background_download_threads_) std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) const
{
return cache_metadata->getFileSegmentPath(key, file_segment.offset(), file_segment.getKind(), user);
}
Poco::Logger * KeyMetadata::logger() const
{
return cache_metadata->log;
}
CacheMetadata::CacheMetadata(
const std::string & path_,
size_t background_download_queue_size_limit_,
size_t background_download_threads_,
bool write_cache_per_user_directory_)
: path(path_) : path(path_)
, cleanup_queue(std::make_shared<CleanupQueue>()) , cleanup_queue(std::make_shared<CleanupQueue>())
, download_queue(std::make_shared<DownloadQueue>(background_download_queue_size_limit_)) , download_queue(std::make_shared<DownloadQueue>(background_download_queue_size_limit_))
, write_cache_per_user_directory(write_cache_per_user_directory_)
, log(&Poco::Logger::get("CacheMetadata")) , log(&Poco::Logger::get("CacheMetadata"))
, download_threads_num(background_download_threads_) , download_threads_num(background_download_threads_)
{ {
@ -157,15 +186,26 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s
return std::to_string(offset) + file_suffix; return std::to_string(offset) + file_suffix;
} }
String CacheMetadata::getPathForFileSegment(const Key & key, size_t offset, FileSegmentKind segment_kind) const String CacheMetadata::getFileSegmentPath(
const Key & key,
size_t offset,
FileSegmentKind segment_kind,
const UserInfo & user) const
{ {
return fs::path(getPathForKey(key)) / getFileNameForFileSegment(offset, segment_kind); return fs::path(getKeyPath(key, user)) / getFileNameForFileSegment(offset, segment_kind);
} }
String CacheMetadata::getPathForKey(const Key & key) const String CacheMetadata::getKeyPath(const Key & key, const UserInfo & user) const
{ {
const auto key_str = key.toString(); if (write_cache_per_user_directory)
return fs::path(path) / key_str.substr(0, 3) / key_str; {
return fs::path(path) / fmt::format("{}.{}", user.user_id, user.weight.value()) / key.toString();
}
else
{
const auto key_str = key.toString();
return fs::path(path) / key_str.substr(0, 3) / key_str;
}
} }
CacheMetadataGuard::Lock CacheMetadata::MetadataBucket::lock() const CacheMetadataGuard::Lock CacheMetadata::MetadataBucket::lock() const
@ -183,9 +223,10 @@ CacheMetadata::MetadataBucket & CacheMetadata::getMetadataBucket(const Key & key
LockedKeyPtr CacheMetadata::lockKeyMetadata( LockedKeyPtr CacheMetadata::lockKeyMetadata(
const FileCacheKey & key, const FileCacheKey & key,
KeyNotFoundPolicy key_not_found_policy, KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load) bool is_initial_load)
{ {
auto key_metadata = getKeyMetadata(key, key_not_found_policy, is_initial_load); auto key_metadata = getKeyMetadata(key, key_not_found_policy, user, is_initial_load);
if (!key_metadata) if (!key_metadata)
return nullptr; return nullptr;
@ -218,12 +259,13 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata(
/// Now we are at the case when the key was removed (key_state == KeyMetadata::KeyState::REMOVED) /// Now we are at the case when the key was removed (key_state == KeyMetadata::KeyState::REMOVED)
/// but we need to return empty key (key_not_found_policy == KeyNotFoundPolicy::CREATE_EMPTY) /// but we need to return empty key (key_not_found_policy == KeyNotFoundPolicy::CREATE_EMPTY)
/// Retry /// Retry
return lockKeyMetadata(key, key_not_found_policy); return lockKeyMetadata(key, key_not_found_policy, user);
} }
KeyMetadataPtr CacheMetadata::getKeyMetadata( KeyMetadataPtr CacheMetadata::getKeyMetadata(
const Key & key, const Key & key,
KeyNotFoundPolicy key_not_found_policy, KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load) bool is_initial_load)
{ {
auto & bucket = getMetadataBucket(key); auto & bucket = getMetadataBucket(key);
@ -240,10 +282,10 @@ KeyMetadataPtr CacheMetadata::getKeyMetadata(
return nullptr; return nullptr;
it = bucket.emplace( it = bucket.emplace(
key, std::make_shared<KeyMetadata>( key, std::make_shared<KeyMetadata>(key, user, this, is_initial_load)).first;
key, getPathForKey(key), cleanup_queue, download_queue, log, key_prefix_directory_mutex, is_initial_load)).first;
} }
it->second->assertAccess(user.user_id);
return it->second; return it->second;
} }
@ -255,13 +297,16 @@ bool CacheMetadata::isEmpty() const
return true; return true;
} }
void CacheMetadata::iterate(IterateFunc && func) void CacheMetadata::iterate(IterateFunc && func, const KeyMetadata::UserID & user_id)
{ {
for (auto & bucket : metadata_buckets) for (auto & bucket : metadata_buckets)
{ {
auto lk = bucket.lock(); auto lk = bucket.lock();
for (auto & [key, key_metadata] : bucket) for (auto & [key, key_metadata] : bucket)
{ {
if (!key_metadata->checkAccess(user_id))
continue;
auto locked_key = key_metadata->lockNoStateCheck(); auto locked_key = key_metadata->lockNoStateCheck();
const auto key_state = locked_key->getKeyState(); const auto key_state = locked_key->getKeyState();
@ -279,13 +324,16 @@ void CacheMetadata::iterate(IterateFunc && func)
} }
} }
void CacheMetadata::removeAllKeys(bool if_releasable) void CacheMetadata::removeAllKeys(bool if_releasable, const UserID & user_id)
{ {
for (auto & bucket : metadata_buckets) for (auto & bucket : metadata_buckets)
{ {
auto lock = bucket.lock(); auto lock = bucket.lock();
for (auto it = bucket.begin(); it != bucket.end();) for (auto it = bucket.begin(); it != bucket.end();)
{ {
if (!it->second->checkAccess(user_id))
continue;
auto locked_key = it->second->lockNoStateCheck(); auto locked_key = it->second->lockNoStateCheck();
if (locked_key->getKeyState() == KeyMetadata::KeyState::ACTIVE) if (locked_key->getKeyState() == KeyMetadata::KeyState::ACTIVE)
{ {
@ -301,7 +349,7 @@ void CacheMetadata::removeAllKeys(bool if_releasable)
} }
} }
void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasable) void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasable, const UserID & user_id)
{ {
auto & bucket = getMetadataBucket(key); auto & bucket = getMetadataBucket(key);
auto lock = bucket.lock(); auto lock = bucket.lock();
@ -314,6 +362,7 @@ void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasabl
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key: {}", key); throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key: {}", key);
} }
it->second->assertAccess(user_id);
auto locked_key = it->second->lockNoStateCheck(); auto locked_key = it->second->lockNoStateCheck();
auto state = locked_key->getKeyState(); auto state = locked_key->getKeyState();
if (state != KeyMetadata::KeyState::ACTIVE) if (state != KeyMetadata::KeyState::ACTIVE)
@ -321,7 +370,8 @@ void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasabl
if (if_exists) if (if_exists)
return; return;
else else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key: {} (state: {})", key, magic_enum::enum_name(state)); throw Exception(ErrorCodes::BAD_ARGUMENTS,
"No such key: {} (state: {})", key, magic_enum::enum_name(state));
} }
bool removed_all = locked_key->removeAllFileSegments(if_releasable); bool removed_all = locked_key->removeAllFileSegments(if_releasable);
@ -346,7 +396,7 @@ CacheMetadata::removeEmptyKey(
LOG_DEBUG(log, "Key {} is removed from metadata", key); LOG_DEBUG(log, "Key {} is removed from metadata", key);
const fs::path key_directory = getPathForKey(key); const fs::path key_directory = getKeyPath(key, locked_key.getKeyMetadata()->user);
const fs::path key_prefix_directory = key_directory.parent_path(); const fs::path key_prefix_directory = key_directory.parent_path();
try try
@ -383,7 +433,7 @@ CacheMetadata::removeEmptyKey(
class CleanupQueue class CleanupQueue
{ {
friend struct CacheMetadata; friend class CacheMetadata;
public: public:
void add(const FileCacheKey & key) void add(const FileCacheKey & key)
{ {
@ -467,7 +517,7 @@ void CacheMetadata::cleanupThreadFunc()
class DownloadQueue class DownloadQueue
{ {
friend struct CacheMetadata; friend class CacheMetadata;
public: public:
explicit DownloadQueue(size_t queue_size_limit_) : queue_size_limit(queue_size_limit_) {} explicit DownloadQueue(size_t queue_size_limit_) : queue_size_limit(queue_size_limit_) {}
@ -504,7 +554,10 @@ private:
struct DownloadInfo struct DownloadInfo
{ {
DownloadInfo(const CacheMetadata::Key & key_, const size_t & offset_, const std::weak_ptr<FileSegment> & file_segment_) DownloadInfo(
const CacheMetadata::Key & key_,
const size_t & offset_,
const std::weak_ptr<FileSegment> & file_segment_)
: key(key_), offset(offset_), file_segment(file_segment_) {} : key(key_), offset(offset_), file_segment(file_segment_) {}
CacheMetadata::Key key; CacheMetadata::Key key;
@ -556,7 +609,7 @@ void CacheMetadata::downloadThreadFunc(const bool & stop_flag)
try try
{ {
{ {
auto locked_key = lockKeyMetadata(key, KeyNotFoundPolicy::RETURN_NULL); auto locked_key = lockKeyMetadata(key, KeyNotFoundPolicy::RETURN_NULL, FileCache::getInternalUser());
if (!locked_key) if (!locked_key)
continue; continue;
@ -616,7 +669,7 @@ bool CacheMetadata::setBackgroundDownloadQueueSizeLimit(size_t size)
return download_queue->setQueueLimit(size); return download_queue->setQueueLimit(size);
} }
void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory) void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory) const
{ {
LOG_TEST( LOG_TEST(
log, "Downloading {} bytes for file segment {}", log, "Downloading {} bytes for file segment {}",
@ -685,7 +738,8 @@ void CacheMetadata::startup()
for (size_t i = 0; i < download_threads_num; ++i) for (size_t i = 0; i < download_threads_num; ++i)
{ {
download_threads.emplace_back(std::make_shared<DownloadThread>()); download_threads.emplace_back(std::make_shared<DownloadThread>());
download_threads.back()->thread = std::make_unique<ThreadFromGlobalPool>([this, thread = download_threads.back()] { downloadThreadFunc(thread->stop_flag); }); download_threads.back()->thread = std::make_unique<ThreadFromGlobalPool>(
[this, thread = download_threads.back()] { downloadThreadFunc(thread->stop_flag); });
} }
cleanup_thread = std::make_unique<ThreadFromGlobalPool>([this]{ cleanupThreadFunc(); }); cleanup_thread = std::make_unique<ThreadFromGlobalPool>([this]{ cleanupThreadFunc(); });
} }
@ -760,6 +814,16 @@ bool CacheMetadata::setBackgroundDownloadThreads(size_t threads_num)
return true; return true;
} }
bool KeyMetadata::addToDownloadQueue(FileSegmentPtr file_segment)
{
return cache_metadata->download_queue->add(file_segment);
}
void KeyMetadata::addToCleanupQueue()
{
cache_metadata->cleanup_queue->add(key);
}
LockedKey::LockedKey(std::shared_ptr<KeyMetadata> key_metadata_) LockedKey::LockedKey(std::shared_ptr<KeyMetadata> key_metadata_)
: key_metadata(key_metadata_) : key_metadata(key_metadata_)
, lock(key_metadata->guard.lock()) , lock(key_metadata->guard.lock())
@ -778,8 +842,8 @@ LockedKey::~LockedKey()
/// See comment near cleanupThreadFunc() for more details. /// See comment near cleanupThreadFunc() for more details.
key_metadata->key_state = KeyMetadata::KeyState::REMOVING; key_metadata->key_state = KeyMetadata::KeyState::REMOVING;
LOG_DEBUG(key_metadata->log, "Submitting key {} for removal", getKey()); LOG_DEBUG(key_metadata->logger(), "Submitting key {} for removal", getKey());
key_metadata->cleanup_queue->add(getKey()); key_metadata->addToCleanupQueue();
} }
void LockedKey::removeFromCleanupQueue() void LockedKey::removeFromCleanupQueue()
@ -841,7 +905,10 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, bool can_be_br
return removeFileSegmentImpl(it, file_segment->lock(), can_be_broken); return removeFileSegmentImpl(it, file_segment->lock(), can_be_broken);
} }
KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegmentGuard::Lock & segment_lock, bool can_be_broken) KeyMetadata::iterator LockedKey::removeFileSegment(
size_t offset,
const FileSegmentGuard::Lock & segment_lock,
bool can_be_broken)
{ {
auto it = key_metadata->find(offset); auto it = key_metadata->find(offset);
if (it == key_metadata->end()) if (it == key_metadata->end())
@ -850,12 +917,15 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm
return removeFileSegmentImpl(it, segment_lock, can_be_broken); return removeFileSegmentImpl(it, segment_lock, can_be_broken);
} }
KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock & segment_lock, bool can_be_broken) KeyMetadata::iterator LockedKey::removeFileSegmentImpl(
KeyMetadata::iterator it,
const FileSegmentGuard::Lock & segment_lock,
bool can_be_broken)
{ {
auto file_segment = it->second->file_segment; auto file_segment = it->second->file_segment;
LOG_DEBUG( LOG_DEBUG(
key_metadata->log, "Remove from cache. Key: {}, offset: {}, size: {}", key_metadata->logger(), "Remove from cache. Key: {}, offset: {}, size: {}",
getKey(), file_segment->offset(), file_segment->reserved_size); getKey(), file_segment->offset(), file_segment->reserved_size);
chassert(can_be_broken || file_segment->assertCorrectnessUnlocked(segment_lock)); chassert(can_be_broken || file_segment->assertCorrectnessUnlocked(segment_lock));
@ -880,14 +950,14 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it,
OpenedFileCache::instance().remove(path, flags); OpenedFileCache::instance().remove(path, flags);
OpenedFileCache::instance().remove(path, flags | O_DIRECT); OpenedFileCache::instance().remove(path, flags | O_DIRECT);
LOG_TEST(key_metadata->log, "Removed file segment at path: {}", path); LOG_TEST(key_metadata->logger(), "Removed file segment at path: {}", path);
} }
else if (file_segment->downloaded_size && !can_be_broken) else if (file_segment->downloaded_size && !can_be_broken)
{ {
#ifdef ABORT_ON_LOGICAL_ERROR #ifdef ABORT_ON_LOGICAL_ERROR
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path);
#else #else
LOG_WARNING(key_metadata->log, "Expected path {} to exist, while removing {}:{}", LOG_WARNING(key_metadata->logger(), "Expected path {} to exist, while removing {}:{}",
path, getKey(), file_segment->offset()); path, getKey(), file_segment->offset());
#endif #endif
} }
@ -942,7 +1012,7 @@ bool LockedKey::addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &
auto it = key_metadata->find(offset); auto it = key_metadata->find(offset);
if (it == key_metadata->end()) if (it == key_metadata->end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is not offset {}", offset); throw Exception(ErrorCodes::LOGICAL_ERROR, "There is not offset {}", offset);
return key_metadata->download_queue->add(it->second->file_segment); return key_metadata->addToDownloadQueue(it->second->file_segment);
} }
std::optional<FileSegment::Range> LockedKey::hasIntersectingRange(const FileSegment::Range & range) const std::optional<FileSegment::Range> LockedKey::hasIntersectingRange(const FileSegment::Range & range) const
@ -1042,7 +1112,7 @@ std::vector<FileSegment::Info> LockedKey::sync()
if (!fs::exists(path)) if (!fs::exists(path))
{ {
LOG_WARNING( LOG_WARNING(
key_metadata->log, key_metadata->logger(),
"File segment has DOWNLOADED state, but file does not exist ({})", "File segment has DOWNLOADED state, but file does not exist ({})",
file_segment->getInfoForLog()); file_segment->getInfoForLog());
@ -1061,7 +1131,7 @@ std::vector<FileSegment::Info> LockedKey::sync()
} }
LOG_WARNING( LOG_WARNING(
key_metadata->log, key_metadata->logger(),
"File segment has unexpected size. Having {}, expected {} ({})", "File segment has unexpected size. Having {}, expected {} ({})",
actual_size, expected_size, file_segment->getInfoForLog()); actual_size, expected_size, file_segment->getInfoForLog());

View File

@ -15,7 +15,9 @@ class CleanupQueue;
using CleanupQueuePtr = std::shared_ptr<CleanupQueue>; using CleanupQueuePtr = std::shared_ptr<CleanupQueue>;
class DownloadQueue; class DownloadQueue;
using DownloadQueuePtr = std::shared_ptr<DownloadQueue>; using DownloadQueuePtr = std::shared_ptr<DownloadQueue>;
using FileSegmentsHolderPtr = std::unique_ptr<FileSegmentsHolder>; using FileSegmentsHolderPtr = std::unique_ptr<FileSegmentsHolder>;
class CacheMetadata;
struct FileSegmentMetadata : private boost::noncopyable struct FileSegmentMetadata : private boost::noncopyable
@ -43,17 +45,18 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
private boost::noncopyable, private boost::noncopyable,
public std::enable_shared_from_this<KeyMetadata> public std::enable_shared_from_this<KeyMetadata>
{ {
friend class CacheMetadata;
friend struct LockedKey; friend struct LockedKey;
using Key = FileCacheKey; using Key = FileCacheKey;
using iterator = iterator; using iterator = iterator;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
KeyMetadata( KeyMetadata(
const Key & key_, const Key & key_,
const std::string & key_path_, const UserInfo & user_id_,
CleanupQueuePtr cleanup_queue_, const CacheMetadata * cache_metadata_,
DownloadQueuePtr download_queue_,
Poco::Logger * log_,
std::shared_mutex & key_prefix_directory_mutex_,
bool created_base_directory_ = false); bool created_base_directory_ = false);
enum class KeyState enum class KeyState
@ -64,19 +67,22 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
}; };
const Key key; const Key key;
const std::string key_path; const UserInfo user;
LockedKeyPtr lock(); LockedKeyPtr lock();
/// Return nullptr if key has non-ACTIVE state.
LockedKeyPtr tryLock(); LockedKeyPtr tryLock();
LockedKeyPtr lockNoStateCheck();
bool createBaseDirectory(); bool createBaseDirectory();
std::string getPath() const;
std::string getFileSegmentPath(const FileSegment & file_segment) const; std::string getFileSegmentPath(const FileSegment & file_segment) const;
bool checkAccess(const UserID & user_id_) const;
void assertAccess(const UserID & user_id_) const;
/// This method is used for loadMetadata() on server startup, /// This method is used for loadMetadata() on server startup,
/// where we know there is no concurrency on Key and we do not want therefore taking a KeyGuard::Lock, /// where we know there is no concurrency on Key and we do not want therefore taking a KeyGuard::Lock,
/// therefore we use this Unlocked version. This method should not be used anywhere else. /// therefore we use this Unlocked version. This method should not be used anywhere else.
@ -85,41 +91,52 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
size_t sizeUnlocked() const { return size(); } size_t sizeUnlocked() const { return size(); }
private: private:
const CacheMetadata * cache_metadata;
KeyState key_state = KeyState::ACTIVE; KeyState key_state = KeyState::ACTIVE;
KeyGuard guard; KeyGuard guard;
const CleanupQueuePtr cleanup_queue;
const DownloadQueuePtr download_queue;
std::shared_mutex & key_prefix_directory_mutex;
std::atomic<bool> created_base_directory = false; std::atomic<bool> created_base_directory = false;
Poco::Logger * log;
LockedKeyPtr lockNoStateCheck();
Poco::Logger * logger() const;
bool addToDownloadQueue(FileSegmentPtr file_segment);
void addToCleanupQueue();
}; };
using KeyMetadataPtr = std::shared_ptr<KeyMetadata>; using KeyMetadataPtr = std::shared_ptr<KeyMetadata>;
struct CacheMetadata class CacheMetadata : private boost::noncopyable
{ {
friend struct KeyMetadata;
public: public:
using Key = FileCacheKey; using Key = FileCacheKey;
using IterateFunc = std::function<void(LockedKey &)>; using IterateFunc = std::function<void(LockedKey &)>;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
explicit CacheMetadata(const std::string & path_, size_t background_download_queue_size_limit_, size_t background_download_threads_); explicit CacheMetadata(
const std::string & path_,
size_t background_download_queue_size_limit_,
size_t background_download_threads_,
bool write_cache_per_user_directory_);
void startup(); void startup();
bool isEmpty() const;
const String & getBaseDirectory() const { return path; } const String & getBaseDirectory() const { return path; }
String getPathForFileSegment( String getKeyPath(const Key & key, const UserInfo & user) const;
String getFileSegmentPath(
const Key & key, const Key & key,
size_t offset, size_t offset,
FileSegmentKind segment_kind) const; FileSegmentKind segment_kind,
const UserInfo & user) const;
String getPathForKey(const Key & key) const; void iterate(IterateFunc && func, const UserID & user_id);
static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind);
void iterate(IterateFunc && func);
bool isEmpty() const;
enum class KeyNotFoundPolicy enum class KeyNotFoundPolicy
{ {
@ -132,15 +149,17 @@ public:
KeyMetadataPtr getKeyMetadata( KeyMetadataPtr getKeyMetadata(
const Key & key, const Key & key,
KeyNotFoundPolicy key_not_found_policy, KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load = false); bool is_initial_load = false);
LockedKeyPtr lockKeyMetadata( LockedKeyPtr lockKeyMetadata(
const Key & key, const Key & key,
KeyNotFoundPolicy key_not_found_policy, KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load = false); bool is_initial_load = false);
void removeKey(const Key & key, bool if_exists, bool if_releasable); void removeKey(const Key & key, bool if_exists, bool if_releasable, const UserID & user_id);
void removeAllKeys(bool if_releasable); void removeAllKeys(bool if_releasable, const UserID & user_id);
void shutdown(); void shutdown();
@ -151,12 +170,15 @@ public:
bool isBackgroundDownloadEnabled(); bool isBackgroundDownloadEnabled();
private: private:
const std::string path; /// Cache base path static constexpr size_t buckets_num = 1024;
const std::string path;
const CleanupQueuePtr cleanup_queue; const CleanupQueuePtr cleanup_queue;
const DownloadQueuePtr download_queue; const DownloadQueuePtr download_queue;
const bool write_cache_per_user_directory;
std::shared_mutex key_prefix_directory_mutex;
Poco::Logger * log; Poco::Logger * log;
mutable std::shared_mutex key_prefix_directory_mutex;
struct MetadataBucket : public std::unordered_map<FileCacheKey, KeyMetadataPtr> struct MetadataBucket : public std::unordered_map<FileCacheKey, KeyMetadataPtr>
{ {
@ -165,7 +187,6 @@ private:
mutable CacheMetadataGuard guard; mutable CacheMetadataGuard guard;
}; };
static constexpr size_t buckets_num = 1024;
std::vector<MetadataBucket> metadata_buckets{buckets_num}; std::vector<MetadataBucket> metadata_buckets{buckets_num};
struct DownloadThread struct DownloadThread
@ -173,13 +194,15 @@ private:
std::unique_ptr<ThreadFromGlobalPool> thread; std::unique_ptr<ThreadFromGlobalPool> thread;
bool stop_flag{false}; bool stop_flag{false};
}; };
std::vector<std::shared_ptr<DownloadThread>> download_threads;
std::atomic<size_t> download_threads_num;
std::atomic<size_t> download_threads_num;
std::vector<std::shared_ptr<DownloadThread>> download_threads;
std::unique_ptr<ThreadFromGlobalPool> cleanup_thread; std::unique_ptr<ThreadFromGlobalPool> cleanup_thread;
static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind);
MetadataBucket & getMetadataBucket(const Key & key); MetadataBucket & getMetadataBucket(const Key & key);
void downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory); void downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory) const;
MetadataBucket::iterator removeEmptyKey( MetadataBucket::iterator removeEmptyKey(
MetadataBucket & bucket, MetadataBucket & bucket,
MetadataBucket::iterator it, MetadataBucket::iterator it,

View File

@ -1,5 +1,6 @@
#include <Interpreters/Cache/QueryLimit.h> #include <Interpreters/Cache/QueryLimit.h>
#include <Interpreters/Cache/Metadata.h> #include <Interpreters/Cache/Metadata.h>
#include <Interpreters/Cache/FileCache.h>
namespace DB namespace DB
{ {
@ -68,9 +69,10 @@ void FileCacheQueryLimit::QueryContext::add(
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const FileCache::UserInfo & user,
const CacheGuard::Lock & lock) const CacheGuard::Lock & lock)
{ {
auto it = getPriority().add(key_metadata, offset, size, lock); auto it = getPriority().add(key_metadata, offset, size, user, lock);
auto [_, inserted] = records.emplace(FileCacheKeyAndOffset{key_metadata->key, offset}, it); auto [_, inserted] = records.emplace(FileCacheKeyAndOffset{key_metadata->key, offset}, it);
if (!inserted) if (!inserted)
{ {
@ -107,4 +109,27 @@ IFileCachePriority::IteratorPtr FileCacheQueryLimit::QueryContext::tryGet(
} }
FileCacheQueryLimit::QueryContextHolder::QueryContextHolder(
const String & query_id_,
FileCache * cache_,
FileCacheQueryLimit * query_limit_,
FileCacheQueryLimit::QueryContextPtr context_)
: query_id(query_id_)
, cache(cache_)
, query_limit(query_limit_)
, context(context_)
{
}
FileCacheQueryLimit::QueryContextHolder::~QueryContextHolder()
{
/// If only the query_map and the current holder hold the context_query,
/// the query has been completed and the query_context is released.
if (context && context.use_count() == 2)
{
auto lock = cache->lockCache();
query_limit->removeQueryContext(query_id, lock);
}
}
} }

View File

@ -44,6 +44,7 @@ public:
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const FileCacheUserInfo & user,
const CacheGuard::Lock &); const CacheGuard::Lock &);
void remove( void remove(
@ -58,6 +59,21 @@ public:
const bool recache_on_query_limit_exceeded; const bool recache_on_query_limit_exceeded;
}; };
struct QueryContextHolder : private boost::noncopyable
{
QueryContextHolder(const String & query_id_, FileCache * cache_, FileCacheQueryLimit * query_limit_, QueryContextPtr context_);
QueryContextHolder() = default;
~QueryContextHolder();
String query_id;
FileCache * cache;
FileCacheQueryLimit * query_limit;
QueryContextPtr context;
};
using QueryContextHolderPtr = std::unique_ptr<QueryContextHolder>;
private: private:
using QueryContextMap = std::unordered_map<String, QueryContextPtr>; using QueryContextMap = std::unordered_map<String, QueryContextPtr>;
QueryContextMap query_map; QueryContextMap query_map;

View File

@ -21,11 +21,13 @@ namespace
SLRUFileCachePriority::SLRUFileCachePriority( SLRUFileCachePriority::SLRUFileCachePriority(
size_t max_size_, size_t max_size_,
size_t max_elements_, size_t max_elements_,
double size_ratio_) double size_ratio_,
LRUFileCachePriority::StatePtr probationary_state_,
LRUFileCachePriority::StatePtr protected_state_)
: IFileCachePriority(max_size_, max_elements_) : IFileCachePriority(max_size_, max_elements_)
, size_ratio(size_ratio_) , size_ratio(size_ratio_)
, protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, size_ratio))) , protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, size_ratio), protected_state_))
, probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - size_ratio))) , probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - size_ratio), probationary_state_))
{ {
LOG_DEBUG( LOG_DEBUG(
log, "Using probationary queue size: {}, protected queue size: {}", log, "Using probationary queue size: {}, protected queue size: {}",
@ -42,15 +44,32 @@ 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);
} }
bool SLRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) const bool SLRUFileCachePriority::canFit( /// NOLINT
size_t size,
const CacheGuard::Lock & lock,
IteratorPtr reservee,
bool best_effort) const
{ {
return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock); if (best_effort)
return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock);
if (reservee)
{
const auto * slru_iterator = assert_cast<SLRUIterator *>(reservee.get());
if (slru_iterator->is_protected)
return protected_queue.canFit(size, lock);
else
return probationary_queue.canFit(size, lock);
}
else
return probationary_queue.canFit(size, lock);
} }
IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( /// NOLINT IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( /// NOLINT
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const UserInfo &,
const CacheGuard::Lock & lock, const CacheGuard::Lock & lock,
bool is_startup) bool is_startup)
{ {
@ -83,13 +102,14 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock & lock) const CacheGuard::Lock & lock)
{ {
/// 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 (!reservee) if (!reservee)
{ {
return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock); return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, user_id, lock);
} }
/// If `it` not nullptr (e.g. is already in some queue), /// If `it` not nullptr (e.g. is already in some queue),
@ -97,7 +117,7 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
/// (in order to know where we need to free space). /// (in order to know where we need to free space).
if (!assert_cast<SLRUIterator *>(reservee.get())->is_protected) if (!assert_cast<SLRUIterator *>(reservee.get())->is_protected)
{ {
return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock); return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, user_id, lock);
} }
/// Entry is in protected queue. /// Entry is in protected queue.
@ -114,13 +134,13 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
FileCacheReserveStat downgrade_stat; FileCacheReserveStat downgrade_stat;
FinalizeEvictionFunc noop; FinalizeEvictionFunc noop;
if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, reservee, noop, lock)) if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, reservee, noop, user_id, 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, reservee, noop, lock)) && !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, reservee, noop, user_id, lock))
return false; return false;
finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable
@ -168,7 +188,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
FileCacheReserveStat downgrade_stat; FileCacheReserveStat downgrade_stat;
FinalizeEvictionFunc noop; FinalizeEvictionFunc noop;
if (!protected_queue.collectCandidatesForEviction(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).
@ -191,7 +211,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
if (size_to_free) if (size_to_free)
{ {
if (!probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock)) if (!probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, {}, lock))
{ {
/// "downgrade" candidates cannot be moved to probationary queue, /// "downgrade" candidates cannot be moved to probationary queue,
/// so entry cannot be moved to protected queue as well. /// so entry cannot be moved to protected queue as well.
@ -222,11 +242,11 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
iterator.is_protected = true; iterator.is_protected = true;
} }
std::vector<FileSegmentInfo> SLRUFileCachePriority::dump(const CacheGuard::Lock & lock) IFileCachePriority::PriorityDumpPtr SLRUFileCachePriority::dump(const CacheGuard::Lock & lock)
{ {
auto res = probationary_queue.dump(lock); auto res = dynamic_pointer_cast<LRUFileCachePriority::LRUPriorityDump>(probationary_queue.dump(lock));
auto part_res = protected_queue.dump(lock); auto part_res = dynamic_pointer_cast<LRUFileCachePriority::LRUPriorityDump>(protected_queue.dump(lock));
res.insert(res.end(), part_res.begin(), part_res.end()); res->merge(*part_res);
return res; return res;
} }
@ -264,7 +284,6 @@ SLRUFileCachePriority::SLRUIterator::SLRUIterator(
SLRUFileCachePriority::EntryPtr SLRUFileCachePriority::SLRUIterator::getEntry() const SLRUFileCachePriority::EntryPtr SLRUFileCachePriority::SLRUIterator::getEntry() const
{ {
chassert(entry == lru_iterator.getEntry());
return entry; return entry;
} }

View File

@ -14,18 +14,28 @@ class SLRUFileCachePriority : public IFileCachePriority
public: public:
class SLRUIterator; class SLRUIterator;
SLRUFileCachePriority(size_t max_size_, size_t max_elements_, double size_ratio_); SLRUFileCachePriority(
size_t max_size_,
size_t max_elements_,
double size_ratio_,
LRUFileCachePriority::StatePtr probationary_state_ = nullptr,
LRUFileCachePriority::StatePtr protected_state_ = nullptr);
size_t getSize(const CacheGuard::Lock & lock) const override; size_t getSize(const CacheGuard::Lock & lock) const override;
size_t getElementsCount(const CacheGuard::Lock &) const override; size_t getElementsCount(const CacheGuard::Lock &) const override;
bool canFit(size_t size, const CacheGuard::Lock &) const override; bool canFit( /// NOLINT
size_t size,
const CacheGuard::Lock &,
IteratorPtr reservee = nullptr,
bool best_effort = false) const override;
IteratorPtr add( /// NOLINT IteratorPtr add( /// NOLINT
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const UserInfo & user,
const CacheGuard::Lock &, const CacheGuard::Lock &,
bool is_startup = false) override; bool is_startup = false) override;
@ -35,11 +45,12 @@ public:
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock &) override; const CacheGuard::Lock &) override;
void shuffle(const CacheGuard::Lock &) override; void shuffle(const CacheGuard::Lock &) override;
std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) override; PriorityDumpPtr dump(const CacheGuard::Lock &) override;
bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override; bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override;

View File

@ -0,0 +1,24 @@
#pragma once
#include <Core/UUID.h>
namespace DB
{
struct FileCacheUserInfo
{
using UserID = std::string;
using Weight = UInt64;
UserID user_id;
std::optional<Weight> weight = std::nullopt;
FileCacheUserInfo() = default;
explicit FileCacheUserInfo(const UserID & user_id_) : user_id(user_id_) {}
FileCacheUserInfo(const UserID & user_id_, const Weight & weight_) : user_id(user_id_), weight(weight_) {}
bool operator ==(const FileCacheUserInfo & other) const { return user_id == other.user_id; }
};
}

View File

@ -19,7 +19,7 @@ namespace ErrorCodes
} }
WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegment * file_segment_) WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegment * file_segment_)
: WriteBufferFromFileDecorator(std::make_unique<WriteBufferFromFile>(file_segment_->getPathInLocalCache())) : WriteBufferFromFileDecorator(std::make_unique<WriteBufferFromFile>(file_segment_->getPath()))
, file_segment(file_segment_) , file_segment(file_segment_)
{ {
} }
@ -27,7 +27,7 @@ WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegment * file_segment_)
WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegmentsHolderPtr segment_holder_) WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegmentsHolderPtr segment_holder_)
: WriteBufferFromFileDecorator( : WriteBufferFromFileDecorator(
segment_holder_->size() == 1 segment_holder_->size() == 1
? std::make_unique<WriteBufferFromFile>(segment_holder_->front().getPathInLocalCache()) ? std::make_unique<WriteBufferFromFile>(segment_holder_->front().getPath())
: throw Exception(ErrorCodes::LOGICAL_ERROR, "WriteBufferToFileSegment can be created only from single segment")) : throw Exception(ErrorCodes::LOGICAL_ERROR, "WriteBufferToFileSegment can be created only from single segment"))
, file_segment(&segment_holder_->front()) , file_segment(&segment_holder_->front())
, segment_holder(std::move(segment_holder_)) , segment_holder(std::move(segment_holder_))
@ -84,7 +84,7 @@ void WriteBufferToFileSegment::nextImpl()
std::unique_ptr<ReadBuffer> WriteBufferToFileSegment::getReadBufferImpl() std::unique_ptr<ReadBuffer> WriteBufferToFileSegment::getReadBufferImpl()
{ {
finalize(); finalize();
return std::make_unique<ReadBufferFromFile>(file_segment->getPathInLocalCache()); return std::make_unique<ReadBufferFromFile>(file_segment->getPath());
} }
WriteBufferToFileSegment::~WriteBufferToFileSegment() WriteBufferToFileSegment::~WriteBufferToFileSegment()

View File

@ -235,6 +235,7 @@ struct ContextSharedPart : boost::noncopyable
String dictionaries_lib_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries. String dictionaries_lib_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries.
String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts. String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts.
String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches. String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches.
String filesystem_cache_user TSA_GUARDED_BY(mutex);
ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings. ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings.
String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request. String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request.
@ -886,6 +887,12 @@ String Context::getFilesystemCachesPath() const
return shared->filesystem_caches_path; return shared->filesystem_caches_path;
} }
String Context::getFilesystemCacheUser() const
{
SharedLockGuard lock(shared->mutex);
return shared->filesystem_cache_user;
}
Strings Context::getWarnings() const Strings Context::getWarnings() const
{ {
Strings common_warnings; Strings common_warnings;
@ -993,6 +1000,12 @@ void Context::setFilesystemCachesPath(const String & path)
shared->filesystem_caches_path = path; shared->filesystem_caches_path = path;
} }
void Context::setFilesystemCacheUser(const String & user)
{
std::lock_guard lock(shared->mutex);
shared->filesystem_cache_user = user;
}
static void setupTmpPath(Poco::Logger * log, const std::string & path) static void setupTmpPath(Poco::Logger * log, const std::string & path)
try try
{ {

View File

@ -529,6 +529,7 @@ public:
String getDictionariesLibPath() const; String getDictionariesLibPath() const;
String getUserScriptsPath() const; String getUserScriptsPath() const;
String getFilesystemCachesPath() const; String getFilesystemCachesPath() const;
String getFilesystemCacheUser() const;
/// A list of warnings about server configuration to place in `system.warnings` table. /// A list of warnings about server configuration to place in `system.warnings` table.
Strings getWarnings() const; Strings getWarnings() const;
@ -540,6 +541,7 @@ public:
void setTempDataOnDisk(TemporaryDataOnDiskScopePtr temp_data_on_disk_); void setTempDataOnDisk(TemporaryDataOnDiskScopePtr temp_data_on_disk_);
void setFilesystemCachesPath(const String & path); void setFilesystemCachesPath(const String & path);
void setFilesystemCacheUser(const String & user);
void setPath(const String & path); void setPath(const String & path);
void setFlagsPath(const String & path); void setFlagsPath(const String & path);

View File

@ -384,22 +384,22 @@ BlockIO InterpreterSystemQuery::execute()
{ {
auto caches = FileCacheFactory::instance().getAll(); auto caches = FileCacheFactory::instance().getAll();
for (const auto & [_, cache_data] : caches) for (const auto & [_, cache_data] : caches)
cache_data->cache->removeAllReleasable(); cache_data->cache->removeAllReleasable(FileCache::getCommonUser().user_id);
} }
else else
{ {
auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name)->cache; auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name)->cache;
if (query.key_to_drop.empty()) if (query.key_to_drop.empty())
{ {
cache->removeAllReleasable(); cache->removeAllReleasable(FileCache::getCommonUser().user_id);
} }
else else
{ {
auto key = FileCacheKey::fromKeyString(query.key_to_drop); auto key = FileCacheKey::fromKeyString(query.key_to_drop);
if (query.offset_to_drop.has_value()) if (query.offset_to_drop.has_value())
cache->removeFileSegment(key, query.offset_to_drop.value()); cache->removeFileSegment(key, query.offset_to_drop.value(), FileCache::getCommonUser().user_id);
else else
cache->removeKey(key); cache->removeKey(key, FileCache::getCommonUser().user_id);
} }
} }
break; break;
@ -424,7 +424,9 @@ BlockIO InterpreterSystemQuery::execute()
for (const auto & file_segment : file_segments) for (const auto & file_segment : file_segments)
{ {
size_t i = 0; size_t i = 0;
const auto path = cache->getPathInLocalCache(file_segment.key, file_segment.offset, file_segment.kind); const auto path = cache->getFileSegmentPath(
file_segment.key, file_segment.offset, file_segment.kind,
FileCache::UserInfo(file_segment.user_id, file_segment.user_weight));
res_columns[i++]->insert(cache_name); res_columns[i++]->insert(cache_name);
res_columns[i++]->insert(path); res_columns[i++]->insert(path);
res_columns[i++]->insert(file_segment.downloaded_size); res_columns[i++]->insert(file_segment.downloaded_size);

View File

@ -105,7 +105,9 @@ FileSegmentsHolderPtr TemporaryDataOnDisk::createCacheFile(size_t max_file_size)
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);
const auto key = FileSegment::Key::random(); const auto key = FileSegment::Key::random();
auto holder = file_cache->set(key, 0, std::max(10_MiB, max_file_size), CreateFileSegmentSettings(FileSegmentKind::Temporary, /* unbounded */ true)); auto holder = file_cache->set(
key, 0, std::max(10_MiB, max_file_size),
CreateFileSegmentSettings(FileSegmentKind::Temporary, /* unbounded */ true), FileCache::getCommonUser());
chassert(holder->size() == 1); chassert(holder->size() == 1);
holder->back().getKeyMetadata()->createBaseDirectory(); holder->back().getKeyMetadata()->createBaseDirectory();
@ -377,7 +379,7 @@ String TemporaryFileStream::getPath() const
if (file) if (file)
return file->getAbsolutePath(); return file->getAbsolutePath();
if (segment_holder && !segment_holder->empty()) if (segment_holder && !segment_holder->empty())
return segment_holder->front().getPathInLocalCache(); return segment_holder->front().getPath();
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryFileStream has no file"); throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryFileStream has no file");
} }

View File

@ -348,7 +348,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
if (!data_part_storage.isDirectory(file_name)) if (!data_part_storage.isDirectory(file_name))
{ {
auto remote_path = data_part_storage.getRemotePath(file_name); auto remote_path = data_part_storage.getRemotePath(file_name);
cache.removePathIfExists(remote_path); cache.removePathIfExists(remote_path, FileCache::getCommonUser().user_id);
} }
} }

View File

@ -30,6 +30,7 @@ ColumnsDescription StorageSystemFilesystemCache::getColumnsDescription()
{"downloaded_size", std::make_shared<DataTypeUInt64>()}, {"downloaded_size", std::make_shared<DataTypeUInt64>()},
{"kind", std::make_shared<DataTypeString>()}, {"kind", std::make_shared<DataTypeString>()},
{"unbound", std::make_shared<DataTypeNumber<UInt8>>()}, {"unbound", std::make_shared<DataTypeNumber<UInt8>>()},
{"user_id", std::make_shared<DataTypeString>()},
{"file_size", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())}, {"file_size", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
}; };
} }
@ -52,10 +53,12 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
res_columns[i++]->insert(cache_name); res_columns[i++]->insert(cache_name);
res_columns[i++]->insert(cache->getBasePath()); res_columns[i++]->insert(cache->getBasePath());
/// Do not use `file_segment->getPathInLocalCache` here because it will lead to nullptr dereference /// Do not use `file_segment->getPath` here because it will lead to nullptr dereference
/// (because file_segments in getSnapshot doesn't have `cache` field set) /// (because file_segments in getSnapshot doesn't have `cache` field set)
const auto path = cache->getPathInLocalCache(file_segment.key, file_segment.offset, file_segment.kind); const auto path = cache->getFileSegmentPath(
file_segment.key, file_segment.offset, file_segment.kind,
FileCache::UserInfo(file_segment.user_id, file_segment.user_weight));
res_columns[i++]->insert(path); res_columns[i++]->insert(path);
res_columns[i++]->insert(file_segment.key.toString()); res_columns[i++]->insert(file_segment.key.toString());
res_columns[i++]->insert(file_segment.range_left); res_columns[i++]->insert(file_segment.range_left);
@ -67,6 +70,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
res_columns[i++]->insert(file_segment.downloaded_size); res_columns[i++]->insert(file_segment.downloaded_size);
res_columns[i++]->insert(toString(file_segment.kind)); res_columns[i++]->insert(toString(file_segment.kind));
res_columns[i++]->insert(file_segment.is_unbound); res_columns[i++]->insert(file_segment.is_unbound);
res_columns[i++]->insert(file_segment.user_id);
std::error_code ec; std::error_code ec;
auto size = fs::file_size(path, ec); auto size = fs::file_size(path, ec);
@ -74,7 +78,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
res_columns[i++]->insert(size); res_columns[i++]->insert(size);
else else
res_columns[i++]->insertDefault(); res_columns[i++]->insertDefault();
}); }, FileCache::getCommonUser().user_id);
} }
} }