mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
Refactoring
This commit is contained in:
parent
9146391e48
commit
c905937ef6
@ -590,6 +590,7 @@
|
||||
M(708, ILLEGAL_STATISTIC) \
|
||||
M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \
|
||||
M(710, FAULT_INJECTED) \
|
||||
M(711, FILECACHE_ACCESS_DENIED) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -48,6 +48,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
|
||||
const String & source_file_path_,
|
||||
const FileCache::Key & cache_key_,
|
||||
FileCachePtr cache_,
|
||||
const FileCacheUserInfo & user_,
|
||||
ImplementationBufferCreator implementation_buffer_creator_,
|
||||
const ReadSettings & settings_,
|
||||
const String & query_id_,
|
||||
@ -70,6 +71,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
|
||||
, implementation_buffer_creator(implementation_buffer_creator_)
|
||||
, query_id(query_id_)
|
||||
, current_buffer_id(getRandomASCIIString(8))
|
||||
, user(user_)
|
||||
, allow_seeks_after_first_read(allow_seeks_after_first_read_)
|
||||
, use_external_buffer(use_external_buffer_)
|
||||
, 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)
|
||||
{
|
||||
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
|
||||
{
|
||||
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();
|
||||
}
|
||||
@ -166,7 +168,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::CachedReadBufferCreateBufferMicroseconds);
|
||||
|
||||
auto path = file_segment.getPathInLocalCache();
|
||||
auto path = file_segment.getPath();
|
||||
if (cache_file_reader)
|
||||
{
|
||||
chassert(cache_file_reader->getFileName() == path);
|
||||
|
@ -1,12 +1,15 @@
|
||||
#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/WriteBufferFromFile.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <Interpreters/FilesystemCacheLog.h>
|
||||
#include <Interpreters/Cache/FileSegment.h>
|
||||
#include <Interpreters/Cache/UserInfo.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -24,8 +27,9 @@ public:
|
||||
|
||||
CachedOnDiskReadBufferFromFile(
|
||||
const String & source_file_path_,
|
||||
const FileCache::Key & cache_key_,
|
||||
const FileCacheKey & cache_key_,
|
||||
FileCachePtr cache_,
|
||||
const FileCacheUserInfo & user_,
|
||||
ImplementationBufferCreator implementation_buffer_creator_,
|
||||
const ReadSettings & settings_,
|
||||
const String & query_id_,
|
||||
@ -102,7 +106,7 @@ private:
|
||||
bool nextFileSegmentsBatch();
|
||||
|
||||
Poco::Logger * log;
|
||||
FileCache::Key cache_key;
|
||||
FileCacheKey cache_key;
|
||||
String source_file_path;
|
||||
|
||||
FileCachePtr cache;
|
||||
@ -145,13 +149,14 @@ private:
|
||||
|
||||
String query_id;
|
||||
String current_buffer_id;
|
||||
FileCacheUserInfo user;
|
||||
|
||||
bool allow_seeks_after_first_read;
|
||||
[[maybe_unused]]bool use_external_buffer;
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::FilesystemCacheReadBuffers};
|
||||
ProfileEvents::Counters current_file_segment_counters;
|
||||
|
||||
FileCache::QueryContextHolderPtr query_context_holder;
|
||||
FileCacheQueryLimit::QueryContextHolderPtr query_context_holder;
|
||||
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log;
|
||||
};
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include "CachedOnDiskWriteBufferFromFile.h"
|
||||
|
||||
#include <Interpreters/Cache/FileCacheFactory.h>
|
||||
#include <Interpreters/Cache/FileSegment.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/Context.h>
|
||||
#include <IO/SwapHelper.h>
|
||||
|
||||
|
||||
@ -25,6 +25,7 @@ namespace ErrorCodes
|
||||
FileSegmentRangeWriter::FileSegmentRangeWriter(
|
||||
FileCache * cache_,
|
||||
const FileSegment::Key & key_,
|
||||
const FileCacheUserInfo & user_,
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log_,
|
||||
const String & query_id_,
|
||||
const String & source_path_)
|
||||
@ -34,6 +35,7 @@ FileSegmentRangeWriter::FileSegmentRangeWriter(
|
||||
, cache_log(cache_log_)
|
||||
, query_id(query_id_)
|
||||
, 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,
|
||||
/// 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);
|
||||
return file_segments->front();
|
||||
}
|
||||
@ -193,7 +195,6 @@ void FileSegmentRangeWriter::completeFileSegment()
|
||||
appendFilesystemCacheLog(file_segment);
|
||||
}
|
||||
|
||||
|
||||
CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
|
||||
std::unique_ptr<WriteBuffer> impl_,
|
||||
FileCachePtr cache_,
|
||||
@ -201,6 +202,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
|
||||
const FileCache::Key & key_,
|
||||
const String & query_id_,
|
||||
const WriteSettings & settings_,
|
||||
const FileCacheUserInfo & user_,
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log_)
|
||||
: WriteBufferFromFileDecorator(std::move(impl_))
|
||||
, log(&Poco::Logger::get("CachedOnDiskWriteBufferFromFile"))
|
||||
@ -208,6 +210,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
|
||||
, source_path(source_path_)
|
||||
, key(key_)
|
||||
, query_id(query_id_)
|
||||
, user(user_)
|
||||
, throw_on_error_from_cache(settings_.throw_on_error_from_cache)
|
||||
, 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.
|
||||
cache_writer.reset();
|
||||
cache->removeKeyIfExists(key);
|
||||
cache->removeKeyIfExists(key, user.user_id);
|
||||
|
||||
throw;
|
||||
}
|
||||
@ -246,7 +249,7 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size, bool t
|
||||
|
||||
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);
|
||||
|
@ -2,7 +2,9 @@
|
||||
|
||||
#include <IO/WriteBufferFromFileDecorator.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>
|
||||
|
||||
namespace Poco
|
||||
@ -25,8 +27,12 @@ class FileSegmentRangeWriter
|
||||
{
|
||||
public:
|
||||
FileSegmentRangeWriter(
|
||||
FileCache * cache_, const FileSegment::Key & key_,
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log_, const String & query_id_, const String & source_path_);
|
||||
FileCache * cache_,
|
||||
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
|
||||
@ -52,6 +58,7 @@ private:
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log;
|
||||
String query_id;
|
||||
String source_path;
|
||||
FileCacheUserInfo user;
|
||||
|
||||
FileSegmentsHolderPtr file_segments;
|
||||
|
||||
@ -71,15 +78,18 @@ public:
|
||||
std::unique_ptr<WriteBuffer> impl_,
|
||||
FileCachePtr cache_,
|
||||
const String & source_path_,
|
||||
const FileCache::Key & key_,
|
||||
const FileCacheKey & key_,
|
||||
const String & query_id_,
|
||||
const WriteSettings & settings_,
|
||||
const FileCacheUserInfo & user_,
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log_);
|
||||
|
||||
void nextImpl() override;
|
||||
|
||||
void finalizeImpl() override;
|
||||
|
||||
bool cachingStopped() const { return cache_in_error_state_or_disabled; }
|
||||
|
||||
private:
|
||||
void cacheData(char * data, size_t size, bool throw_on_error);
|
||||
|
||||
@ -87,10 +97,11 @@ private:
|
||||
|
||||
FileCachePtr cache;
|
||||
String source_path;
|
||||
FileCache::Key key;
|
||||
FileCacheKey key;
|
||||
|
||||
size_t current_download_offset = 0;
|
||||
const String query_id;
|
||||
const FileCacheUserInfo user;
|
||||
|
||||
bool throw_on_error_from_cache;
|
||||
bool cache_in_error_state_or_disabled = false;
|
||||
|
@ -82,6 +82,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
|
||||
object_path,
|
||||
cache_key,
|
||||
settings.remote_fs_cache,
|
||||
FileCache::getCommonUser(),
|
||||
std::move(current_read_buffer_creator),
|
||||
settings,
|
||||
query_id,
|
||||
|
@ -113,6 +113,7 @@ std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// N
|
||||
key,
|
||||
CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "",
|
||||
modified_write_settings,
|
||||
FileCache::getCommonUser(),
|
||||
Context::getGlobalContextInstance()->getFilesystemCacheLog());
|
||||
}
|
||||
|
||||
@ -125,7 +126,7 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c
|
||||
return;
|
||||
|
||||
/// 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)
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <base/hex.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
@ -28,9 +29,17 @@ namespace ProfileEvents
|
||||
extern const Event FilesystemCacheGetMicroseconds;
|
||||
}
|
||||
|
||||
namespace
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
size_t roundDownToMultiple(size_t num, size_t multiple)
|
||||
{
|
||||
return (num / multiple) * multiple;
|
||||
@ -41,14 +50,12 @@ 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;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
void FileCacheReserveStat::update(size_t size, FileSegmentKind kind, bool releasable)
|
||||
@ -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)
|
||||
, boundary_alignment(settings.boundary_alignment)
|
||||
, 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 + ")"))
|
||||
, 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")
|
||||
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);
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
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
|
||||
@ -276,9 +296,10 @@ std::vector<FileSegment::Range> FileCache::splitRange(size_t offset, size_t size
|
||||
size_t remaining_size = size;
|
||||
|
||||
FileSegments file_segments;
|
||||
const size_t max_size = max_file_segment_size.load();
|
||||
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);
|
||||
|
||||
remaining_size -= current_file_segment_size;
|
||||
@ -305,9 +326,10 @@ FileSegments FileCache::splitRangeIntoFileSegments(
|
||||
size_t remaining_size = size;
|
||||
|
||||
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))
|
||||
{
|
||||
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;
|
||||
|
||||
auto file_segment_metadata_it = addFileSegment(
|
||||
@ -468,11 +490,12 @@ FileSegmentsHolderPtr FileCache::set(
|
||||
const Key & key,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const CreateFileSegmentSettings & create_settings)
|
||||
const CreateFileSegmentSettings & create_settings,
|
||||
const UserInfo & user)
|
||||
{
|
||||
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);
|
||||
|
||||
auto file_segments = getImpl(*locked_key, range, /* file_segments_limit */0);
|
||||
@ -502,7 +525,8 @@ FileCache::getOrSet(
|
||||
size_t size,
|
||||
size_t file_size,
|
||||
const CreateFileSegmentSettings & create_settings,
|
||||
size_t file_segments_limit)
|
||||
size_t file_segments_limit,
|
||||
const UserInfo & user)
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds);
|
||||
|
||||
@ -516,7 +540,7 @@ FileCache::getOrSet(
|
||||
chassert(aligned_offset <= range.left);
|
||||
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.
|
||||
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));
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
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)
|
||||
{
|
||||
FileSegment::Range range(offset, offset + size - 1);
|
||||
@ -705,7 +734,7 @@ KeyMetadata::iterator FileCache::addFileSegment(
|
||||
auto & stash_records = stash->records;
|
||||
|
||||
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))
|
||||
stash->queue->pop(*lock);
|
||||
@ -738,7 +767,11 @@ KeyMetadata::iterator FileCache::addFileSegment(
|
||||
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);
|
||||
|
||||
@ -780,7 +813,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
|
||||
|
||||
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;
|
||||
|
||||
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();
|
||||
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;
|
||||
|
||||
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),
|
||||
/// 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);
|
||||
}
|
||||
|
||||
@ -825,7 +860,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
|
||||
if (query_queue_it)
|
||||
query_queue_it->updateSize(size);
|
||||
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))
|
||||
@ -834,40 +869,40 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
|
||||
return true;
|
||||
}
|
||||
|
||||
void FileCache::iterate(IterateFunc && func)
|
||||
void FileCache::iterate(IterateFunc && func, const UserID & user_id)
|
||||
{
|
||||
return metadata.iterate([&](const LockedKey & locked_key)
|
||||
{
|
||||
for (const auto & file_segment_metadata : locked_key)
|
||||
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();
|
||||
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();
|
||||
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();
|
||||
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);
|
||||
}
|
||||
|
||||
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();
|
||||
|
||||
@ -875,7 +910,7 @@ void FileCache::removeAllReleasable()
|
||||
assertCacheCorrectness();
|
||||
#endif
|
||||
|
||||
metadata.removeAllKeys(/* if_releasable */true);
|
||||
metadata.removeAllKeys(/* if_releasable */true, user_id);
|
||||
|
||||
if (stash)
|
||||
{
|
||||
@ -1002,6 +1037,24 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
|
||||
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;
|
||||
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());
|
||||
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)
|
||||
{
|
||||
@ -1072,9 +1125,9 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
|
||||
auto lock = lockCache();
|
||||
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)
|
||||
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
|
||||
/// 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)
|
||||
{
|
||||
metadata.removeKey(key, false, false);
|
||||
metadata.removeKey(key, false, false, getInternalUser().user_id);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1147,7 +1200,7 @@ void FileCache::deactivateBackgroundOperations()
|
||||
metadata.shutdown();
|
||||
}
|
||||
|
||||
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos()
|
||||
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos(const UserID & user_id)
|
||||
{
|
||||
assertInitialized();
|
||||
#ifndef NDEBUG
|
||||
@ -1159,20 +1212,20 @@ std::vector<FileSegment::Info> FileCache::getFileSegmentInfos()
|
||||
{
|
||||
for (const auto & [_, file_segment_metadata] : locked_key)
|
||||
file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment));
|
||||
});
|
||||
}, user_id);
|
||||
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;
|
||||
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)
|
||||
file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment));
|
||||
return file_segments;
|
||||
}
|
||||
|
||||
std::vector<FileSegment::Info> FileCache::dumpQueue()
|
||||
IFileCachePriority::PriorityDumpPtr FileCache::dumpQueue()
|
||||
{
|
||||
assertInitialized();
|
||||
return main_priority->dump(lockCache());
|
||||
@ -1182,7 +1235,7 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
|
||||
{
|
||||
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)
|
||||
return {};
|
||||
|
||||
@ -1190,8 +1243,9 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
|
||||
|
||||
for (const auto & [offset, file_segment_metadata] : *locked_key)
|
||||
{
|
||||
if (file_segment_metadata->file_segment->state() == FileSegment::State::DOWNLOADED)
|
||||
cache_paths.push_back(metadata.getPathForFileSegment(key, offset, file_segment_metadata->file_segment->getKind()));
|
||||
const auto & file_segment = *file_segment_metadata->file_segment;
|
||||
if (file_segment.state() == FileSegment::State::DOWNLOADED)
|
||||
cache_paths.push_back(locked_key->getKeyMetadata()->getFileSegmentPath(file_segment));
|
||||
}
|
||||
return cache_paths;
|
||||
}
|
||||
@ -1214,7 +1268,7 @@ void FileCache::assertCacheCorrectness()
|
||||
{
|
||||
chassert(file_segment_metadata->file_segment->assertCorrectness());
|
||||
}
|
||||
});
|
||||
}, getInternalUser().user_id);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
FileCache::QueryContextHolder::QueryContextHolder(
|
||||
const String & query_id_,
|
||||
FileCache * cache_,
|
||||
FileCacheQueryLimit::QueryContextPtr context_)
|
||||
: query_id(query_id_)
|
||||
, cache(cache_)
|
||||
, context(context_)
|
||||
if (new_settings.max_file_segment_size != actual_settings.max_file_segment_size)
|
||||
{
|
||||
}
|
||||
|
||||
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();
|
||||
cache->query_limit->removeQueryContext(query_id, lock);
|
||||
max_file_segment_size = actual_settings.max_file_segment_size = new_settings.max_file_segment_size;
|
||||
}
|
||||
}
|
||||
|
||||
@ -1317,7 +1355,7 @@ FileCache::QueryContextHolderPtr FileCache::getQueryContextHolder(
|
||||
|
||||
auto lock = lockCache();
|
||||
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()
|
||||
@ -1327,7 +1365,7 @@ std::vector<FileSegment::Info> FileCache::sync()
|
||||
{
|
||||
auto broken = locked_key.sync();
|
||||
file_segments.insert(file_segments.end(), broken.begin(), broken.end());
|
||||
});
|
||||
}, getInternalUser().user_id);
|
||||
return file_segments;
|
||||
}
|
||||
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Interpreters/Cache/QueryLimit.h>
|
||||
#include <Interpreters/Cache/FileCache_fwd_internal.h>
|
||||
#include <Interpreters/Cache/FileCacheSettings.h>
|
||||
#include <Interpreters/Cache/UserInfo.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
@ -34,12 +35,29 @@ struct FileCacheReserveStat
|
||||
|
||||
size_t non_releasable_size = 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;
|
||||
std::unordered_map<FileSegmentKind, Stat> stat_by_kind;
|
||||
|
||||
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.
|
||||
@ -51,6 +69,9 @@ public:
|
||||
using QueryLimit = DB::FileCacheQueryLimit;
|
||||
using Priority = IFileCachePriority;
|
||||
using PriorityEntry = IFileCachePriority::Entry;
|
||||
using QueryContextHolder = FileCacheQueryLimit::QueryContextHolder;
|
||||
using UserInfo = FileCacheUserInfo;
|
||||
using UserID = UserInfo::UserID;
|
||||
|
||||
FileCache(const std::string & cache_name, const FileCacheSettings & settings);
|
||||
|
||||
@ -62,9 +83,13 @@ public:
|
||||
|
||||
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,
|
||||
@ -83,7 +108,8 @@ public:
|
||||
size_t size,
|
||||
size_t file_size,
|
||||
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
|
||||
@ -94,24 +120,34 @@ public:
|
||||
* with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change
|
||||
* 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.
|
||||
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.
|
||||
void removeKey(const Key & key);
|
||||
void removeKey(const Key & key, const UserID & user_id);
|
||||
|
||||
/// Remove files by `key`.
|
||||
void removeKeyIfExists(const Key & key);
|
||||
void removeKeyIfExists(const Key & key, const UserID & user_id);
|
||||
|
||||
/// Removes files by `path`.
|
||||
void removePathIfExists(const String & path);
|
||||
void removePathIfExists(const String & path, const UserID & user_id);
|
||||
|
||||
/// Remove files by `key`.
|
||||
void removeAllReleasable();
|
||||
void removeAllReleasable(const UserID & user_id);
|
||||
|
||||
std::vector<String> tryGetCachePaths(const Key & key);
|
||||
|
||||
@ -121,48 +157,41 @@ public:
|
||||
|
||||
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();
|
||||
|
||||
/// 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;
|
||||
|
||||
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 &)>;
|
||||
void iterate(IterateFunc && func);
|
||||
void iterate(IterateFunc && func, const UserID & user_id);
|
||||
|
||||
void applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings);
|
||||
|
||||
private:
|
||||
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 boundary_alignment;
|
||||
size_t load_metadata_threads;
|
||||
const bool write_cache_per_user_directory;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
|
@ -21,6 +21,7 @@ struct FileCacheKey
|
||||
static FileCacheKey random();
|
||||
|
||||
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);
|
||||
};
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/NamedCollections/NamedCollections.h>
|
||||
#include <boost/algorithm/string/case_conv.hpp>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -74,6 +75,9 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin
|
||||
|
||||
if (has("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)
|
||||
|
@ -33,6 +33,8 @@ struct FileCacheSettings
|
||||
|
||||
size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS;
|
||||
|
||||
bool write_cache_per_user_id_directory = false;
|
||||
|
||||
std::string cache_policy = "LRU";
|
||||
double slru_size_ratio = 0.5;
|
||||
|
||||
|
@ -85,7 +85,7 @@ FileSegment::FileSegment(
|
||||
case (State::DOWNLOADED):
|
||||
{
|
||||
reserved_size = downloaded_size = size_;
|
||||
chassert(fs::file_size(getPathInLocalCache()) == size_);
|
||||
chassert(fs::file_size(getPath()) == size_);
|
||||
chassert(queue_iterator);
|
||||
chassert(key_metadata.lock());
|
||||
break;
|
||||
@ -115,12 +115,12 @@ FileSegment::State FileSegment::state() const
|
||||
return download_state;
|
||||
}
|
||||
|
||||
String FileSegment::getPathInLocalCache() const
|
||||
String FileSegment::getPath() const
|
||||
{
|
||||
return getKeyMetadata()->getFileSegmentPath(*this);
|
||||
}
|
||||
|
||||
String FileSegment::tryGetPathInLocalCache() const
|
||||
String FileSegment::tryGetPath() const
|
||||
{
|
||||
auto metadata = tryGetKeyMetadata();
|
||||
if (!metadata)
|
||||
@ -182,7 +182,7 @@ void FileSegment::setDownloadedSize(size_t delta)
|
||||
{
|
||||
auto lock = lockFileSegment();
|
||||
downloaded_size += delta;
|
||||
assert(downloaded_size == std::filesystem::file_size(getPathInLocalCache()));
|
||||
assert(downloaded_size == std::filesystem::file_size(getPath()));
|
||||
}
|
||||
|
||||
bool FileSegment::isDownloaded() const
|
||||
@ -339,7 +339,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
|
||||
if (!size)
|
||||
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();
|
||||
@ -531,7 +531,7 @@ bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve
|
||||
if (!reserve_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)
|
||||
setDownloadFailedUnlocked(lockFileSegment());
|
||||
@ -554,7 +554,7 @@ void FileSegment::setDownloadedUnlocked(const FileSegmentGuard::Lock &)
|
||||
}
|
||||
|
||||
chassert(downloaded_size > 0);
|
||||
chassert(fs::file_size(getPathInLocalCache()) == downloaded_size);
|
||||
chassert(fs::file_size(getPath()) == downloaded_size);
|
||||
}
|
||||
|
||||
void FileSegment::setDownloadFailed()
|
||||
@ -654,7 +654,7 @@ void FileSegment::complete()
|
||||
case State::DOWNLOADED:
|
||||
{
|
||||
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(!remote_file_reader);
|
||||
break;
|
||||
@ -800,7 +800,7 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons
|
||||
chassert(downloaded_size == reserved_size);
|
||||
chassert(downloaded_size == range().size());
|
||||
chassert(downloaded_size > 0);
|
||||
chassert(std::filesystem::file_size(getPathInLocalCache()) > 0);
|
||||
chassert(std::filesystem::file_size(getPath()) > 0);
|
||||
chassert(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)
|
||||
{
|
||||
auto lock = file_segment->lockFileSegment();
|
||||
auto key_metadata = file_segment->tryGetKeyMetadata();
|
||||
return Info{
|
||||
.key = file_segment->key(),
|
||||
.offset = file_segment->offset(),
|
||||
.path = file_segment->tryGetPathInLocalCache(),
|
||||
.path = file_segment->tryGetPath(),
|
||||
.range_left = file_segment->range().left,
|
||||
.range_right = file_segment->range().right,
|
||||
.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()),
|
||||
.is_unbound = file_segment->is_unbound,
|
||||
.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);
|
||||
key_metadata.reset();
|
||||
cache = nullptr;
|
||||
queue_iterator = nullptr;
|
||||
try
|
||||
{
|
||||
|
@ -112,7 +112,7 @@ public:
|
||||
|
||||
bool isUnbound() const { return is_unbound; }
|
||||
|
||||
String getPathInLocalCache() const;
|
||||
String getPath() const;
|
||||
|
||||
int getFlagsForLocalRead() const { return O_RDONLY | O_CLOEXEC; }
|
||||
|
||||
@ -243,7 +243,7 @@ private:
|
||||
LockedKeyPtr lockKeyMetadata(bool assert_exists = true) const;
|
||||
FileSegmentGuard::Lock lockFileSegment() const;
|
||||
|
||||
String tryGetPathInLocalCache() const;
|
||||
String tryGetPath() const;
|
||||
|
||||
Key file_key;
|
||||
Range segment_range;
|
||||
|
@ -78,5 +78,7 @@ namespace DB
|
||||
uint64_t references;
|
||||
bool is_unbound;
|
||||
FileCacheQueueEntryType queue_entry_type;
|
||||
std::string user_id;
|
||||
uint64_t user_weight;
|
||||
};
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/Cache/Guards.h>
|
||||
#include <Interpreters/Cache/IFileCachePriority.h>
|
||||
#include <Interpreters/Cache/FileCache_fwd_internal.h>
|
||||
#include <Interpreters/Cache/UserInfo.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -18,6 +19,8 @@ class IFileCachePriority : private boost::noncopyable
|
||||
public:
|
||||
using Key = FileCacheKey;
|
||||
using QueueEntryType = FileCacheQueueEntryType;
|
||||
using UserInfo = FileCacheUserInfo;
|
||||
using UserID = UserInfo::UserID;
|
||||
|
||||
struct Entry
|
||||
{
|
||||
@ -52,8 +55,6 @@ public:
|
||||
};
|
||||
using IteratorPtr = std::shared_ptr<Iterator>;
|
||||
|
||||
IFileCachePriority(size_t max_size_, size_t max_elements_);
|
||||
|
||||
virtual ~IFileCachePriority() = default;
|
||||
|
||||
size_t getElementsLimit(const CacheGuard::Lock &) const { return max_elements; }
|
||||
@ -69,14 +70,28 @@ public:
|
||||
KeyMetadataPtr key_metadata,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const UserInfo & user,
|
||||
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 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)>;
|
||||
virtual bool collectCandidatesForEviction(
|
||||
@ -85,11 +100,14 @@ public:
|
||||
EvictionCandidates & res,
|
||||
IFileCachePriority::IteratorPtr reservee,
|
||||
FinalizeEvictionFunc & finalize_eviction_func,
|
||||
const UserID & user_id,
|
||||
const CacheGuard::Lock &) = 0;
|
||||
|
||||
virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) = 0;
|
||||
|
||||
protected:
|
||||
IFileCachePriority(size_t max_size_, size_t max_elements_);
|
||||
|
||||
size_t max_size = 0;
|
||||
size_t max_elements = 0;
|
||||
};
|
||||
|
@ -29,12 +29,22 @@ namespace ErrorCodes
|
||||
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
|
||||
KeyMetadataPtr key_metadata,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const UserInfo &,
|
||||
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));
|
||||
}
|
||||
@ -63,12 +73,12 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(EntryPtr entry, cons
|
||||
#endif
|
||||
|
||||
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(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"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);
|
||||
@ -102,13 +112,13 @@ LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::
|
||||
|
||||
void LRUFileCachePriority::updateSize(int64_t size)
|
||||
{
|
||||
current_size += size;
|
||||
state->current_size += size;
|
||||
CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size);
|
||||
}
|
||||
|
||||
void LRUFileCachePriority::updateElementsCount(int64_t num)
|
||||
{
|
||||
current_elements_num += num;
|
||||
state->current_elements_num += 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);
|
||||
}
|
||||
@ -200,8 +214,8 @@ bool LRUFileCachePriority::canFit(
|
||||
size_t released_elements_assumption,
|
||||
const CacheGuard::Lock &) const
|
||||
{
|
||||
return (max_size == 0 || (current_size + size - released_size_assumption <= max_size))
|
||||
&& (max_elements == 0 || current_elements_num + 1 - released_elements_assumption <= max_elements);
|
||||
return (max_size == 0 || (state->current_size + size - released_size_assumption <= max_size))
|
||||
&& (max_elements == 0 || state->current_elements_num + 1 - released_elements_assumption <= max_elements);
|
||||
}
|
||||
|
||||
bool LRUFileCachePriority::collectCandidatesForEviction(
|
||||
@ -210,6 +224,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
|
||||
EvictionCandidates & res,
|
||||
IFileCachePriority::IteratorPtr,
|
||||
FinalizeEvictionFunc &,
|
||||
const UserID &,
|
||||
const CacheGuard::Lock & lock)
|
||||
{
|
||||
if (canFit(size, lock))
|
||||
@ -282,7 +297,7 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, L
|
||||
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;
|
||||
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));
|
||||
return IterationResult::CONTINUE;
|
||||
}, lock);
|
||||
return res;
|
||||
return std::make_shared<LRUPriorityDump>(res);
|
||||
}
|
||||
|
||||
bool LRUFileCachePriority::modifySizeLimits(
|
||||
@ -301,8 +316,8 @@ bool LRUFileCachePriority::modifySizeLimits(
|
||||
|
||||
auto check_limits_satisfied = [&]()
|
||||
{
|
||||
return (max_size_ == 0 || current_size <= max_size_)
|
||||
&& (max_elements_ == 0 || current_elements_num <= max_elements_);
|
||||
return (max_size_ == 0 || state->current_size <= max_size_)
|
||||
&& (max_elements_ == 0 || state->current_elements_num <= max_elements_);
|
||||
};
|
||||
|
||||
if (check_limits_satisfied())
|
||||
|
@ -13,26 +13,35 @@ namespace DB
|
||||
/// the head of the queue, and the record with the highest priority is stored at the tail.
|
||||
class LRUFileCachePriority final : public IFileCachePriority
|
||||
{
|
||||
private:
|
||||
class LRUIterator;
|
||||
using LRUQueue = std::list<EntryPtr>;
|
||||
friend class SLRUFileCachePriority;
|
||||
friend class OvercommitFileCachePriority;
|
||||
protected:
|
||||
struct State
|
||||
{
|
||||
std::atomic<size_t> current_size = 0;
|
||||
std::atomic<size_t> current_elements_num = 0;
|
||||
};
|
||||
using StatePtr = std::shared_ptr<State>;
|
||||
|
||||
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
|
||||
KeyMetadataPtr key_metadata,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const UserInfo & user,
|
||||
const CacheGuard::Lock &,
|
||||
bool is_startup = false) override;
|
||||
bool best_effort = false) override;
|
||||
|
||||
bool collectCandidatesForEviction(
|
||||
size_t size,
|
||||
@ -40,27 +49,34 @@ public:
|
||||
EvictionCandidates & res,
|
||||
IFileCachePriority::IteratorPtr reservee,
|
||||
FinalizeEvictionFunc & finalize_eviction_func,
|
||||
const UserID & user_id,
|
||||
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); }
|
||||
|
||||
bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override;
|
||||
|
||||
private:
|
||||
void updateElementsCount(int64_t num);
|
||||
void updateSize(int64_t size);
|
||||
class LRUIterator;
|
||||
using LRUQueue = std::list<EntryPtr>;
|
||||
friend class SLRUFileCachePriority;
|
||||
|
||||
LRUQueue queue;
|
||||
Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority");
|
||||
StatePtr state;
|
||||
|
||||
std::atomic<size_t> current_size = 0;
|
||||
/// current_elements_num is not always equal to queue.size()
|
||||
/// because of invalidated entries.
|
||||
std::atomic<size_t> current_elements_num = 0;
|
||||
void updateElementsCount(int64_t num);
|
||||
void updateSize(int64_t size);
|
||||
|
||||
bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const;
|
||||
|
||||
|
@ -26,6 +26,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int FILECACHE_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_)
|
||||
@ -58,22 +59,36 @@ size_t FileSegmentMetadata::size() const
|
||||
|
||||
KeyMetadata::KeyMetadata(
|
||||
const Key & key_,
|
||||
const std::string & key_path_,
|
||||
CleanupQueuePtr cleanup_queue_,
|
||||
DownloadQueuePtr download_queue_,
|
||||
Poco::Logger * log_,
|
||||
std::shared_mutex & key_prefix_directory_mutex_,
|
||||
const UserInfo & user_,
|
||||
const CacheMetadata * cache_metadata_,
|
||||
bool created_base_directory_)
|
||||
: key(key_)
|
||||
, key_path(key_path_)
|
||||
, cleanup_queue(cleanup_queue_)
|
||||
, download_queue(download_queue_)
|
||||
, key_prefix_directory_mutex(key_prefix_directory_mutex_)
|
||||
, user(user_)
|
||||
, cache_metadata(cache_metadata_)
|
||||
, created_base_directory(created_base_directory_)
|
||||
, log(log_)
|
||||
{
|
||||
if (created_base_directory)
|
||||
chassert(fs::exists(key_path));
|
||||
if (user_ == FileCache::getInternalUser())
|
||||
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()
|
||||
@ -108,8 +123,8 @@ bool KeyMetadata::createBaseDirectory()
|
||||
{
|
||||
try
|
||||
{
|
||||
std::shared_lock lock(key_prefix_directory_mutex);
|
||||
fs::create_directories(key_path);
|
||||
std::shared_lock lock(cache_metadata->key_prefix_directory_mutex);
|
||||
fs::create_directories(getPath());
|
||||
}
|
||||
catch (const fs::filesystem_error & e)
|
||||
{
|
||||
@ -117,7 +132,7 @@ bool KeyMetadata::createBaseDirectory()
|
||||
|
||||
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);
|
||||
|
||||
return false;
|
||||
@ -128,16 +143,30 @@ bool KeyMetadata::createBaseDirectory()
|
||||
return true;
|
||||
}
|
||||
|
||||
std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) const
|
||||
std::string KeyMetadata::getPath() const
|
||||
{
|
||||
return fs::path(key_path)
|
||||
/ CacheMetadata::getFileNameForFileSegment(file_segment.offset(), file_segment.getKind());
|
||||
return cache_metadata->getKeyPath(key, user);
|
||||
}
|
||||
|
||||
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_)
|
||||
, cleanup_queue(std::make_shared<CleanupQueue>())
|
||||
, 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"))
|
||||
, download_threads_num(background_download_threads_)
|
||||
{
|
||||
@ -157,16 +186,27 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s
|
||||
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
|
||||
{
|
||||
if (write_cache_per_user_directory)
|
||||
{
|
||||
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
|
||||
{
|
||||
@ -183,9 +223,10 @@ CacheMetadata::MetadataBucket & CacheMetadata::getMetadataBucket(const Key & key
|
||||
LockedKeyPtr CacheMetadata::lockKeyMetadata(
|
||||
const FileCacheKey & key,
|
||||
KeyNotFoundPolicy key_not_found_policy,
|
||||
const UserInfo & user,
|
||||
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)
|
||||
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)
|
||||
/// but we need to return empty key (key_not_found_policy == KeyNotFoundPolicy::CREATE_EMPTY)
|
||||
/// Retry
|
||||
return lockKeyMetadata(key, key_not_found_policy);
|
||||
return lockKeyMetadata(key, key_not_found_policy, user);
|
||||
}
|
||||
|
||||
KeyMetadataPtr CacheMetadata::getKeyMetadata(
|
||||
const Key & key,
|
||||
KeyNotFoundPolicy key_not_found_policy,
|
||||
const UserInfo & user,
|
||||
bool is_initial_load)
|
||||
{
|
||||
auto & bucket = getMetadataBucket(key);
|
||||
@ -240,10 +282,10 @@ KeyMetadataPtr CacheMetadata::getKeyMetadata(
|
||||
return nullptr;
|
||||
|
||||
it = bucket.emplace(
|
||||
key, std::make_shared<KeyMetadata>(
|
||||
key, getPathForKey(key), cleanup_queue, download_queue, log, key_prefix_directory_mutex, is_initial_load)).first;
|
||||
key, std::make_shared<KeyMetadata>(key, user, this, is_initial_load)).first;
|
||||
}
|
||||
|
||||
it->second->assertAccess(user.user_id);
|
||||
return it->second;
|
||||
}
|
||||
|
||||
@ -255,13 +297,16 @@ bool CacheMetadata::isEmpty() const
|
||||
return true;
|
||||
}
|
||||
|
||||
void CacheMetadata::iterate(IterateFunc && func)
|
||||
void CacheMetadata::iterate(IterateFunc && func, const KeyMetadata::UserID & user_id)
|
||||
{
|
||||
for (auto & bucket : metadata_buckets)
|
||||
{
|
||||
auto lk = bucket.lock();
|
||||
for (auto & [key, key_metadata] : bucket)
|
||||
{
|
||||
if (!key_metadata->checkAccess(user_id))
|
||||
continue;
|
||||
|
||||
auto locked_key = key_metadata->lockNoStateCheck();
|
||||
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)
|
||||
{
|
||||
auto lock = bucket.lock();
|
||||
for (auto it = bucket.begin(); it != bucket.end();)
|
||||
{
|
||||
if (!it->second->checkAccess(user_id))
|
||||
continue;
|
||||
|
||||
auto locked_key = it->second->lockNoStateCheck();
|
||||
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 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);
|
||||
}
|
||||
|
||||
it->second->assertAccess(user_id);
|
||||
auto locked_key = it->second->lockNoStateCheck();
|
||||
auto state = locked_key->getKeyState();
|
||||
if (state != KeyMetadata::KeyState::ACTIVE)
|
||||
@ -321,7 +370,8 @@ void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasabl
|
||||
if (if_exists)
|
||||
return;
|
||||
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);
|
||||
@ -346,7 +396,7 @@ CacheMetadata::removeEmptyKey(
|
||||
|
||||
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();
|
||||
|
||||
try
|
||||
@ -383,7 +433,7 @@ CacheMetadata::removeEmptyKey(
|
||||
|
||||
class CleanupQueue
|
||||
{
|
||||
friend struct CacheMetadata;
|
||||
friend class CacheMetadata;
|
||||
public:
|
||||
void add(const FileCacheKey & key)
|
||||
{
|
||||
@ -467,7 +517,7 @@ void CacheMetadata::cleanupThreadFunc()
|
||||
|
||||
class DownloadQueue
|
||||
{
|
||||
friend struct CacheMetadata;
|
||||
friend class CacheMetadata;
|
||||
public:
|
||||
explicit DownloadQueue(size_t queue_size_limit_) : queue_size_limit(queue_size_limit_) {}
|
||||
|
||||
@ -504,7 +554,10 @@ private:
|
||||
|
||||
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_) {}
|
||||
|
||||
CacheMetadata::Key key;
|
||||
@ -556,7 +609,7 @@ void CacheMetadata::downloadThreadFunc(const bool & stop_flag)
|
||||
try
|
||||
{
|
||||
{
|
||||
auto locked_key = lockKeyMetadata(key, KeyNotFoundPolicy::RETURN_NULL);
|
||||
auto locked_key = lockKeyMetadata(key, KeyNotFoundPolicy::RETURN_NULL, FileCache::getInternalUser());
|
||||
if (!locked_key)
|
||||
continue;
|
||||
|
||||
@ -616,7 +669,7 @@ bool CacheMetadata::setBackgroundDownloadQueueSizeLimit(size_t 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, "Downloading {} bytes for file segment {}",
|
||||
@ -685,7 +738,8 @@ void CacheMetadata::startup()
|
||||
for (size_t i = 0; i < download_threads_num; ++i)
|
||||
{
|
||||
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(); });
|
||||
}
|
||||
@ -760,6 +814,16 @@ bool CacheMetadata::setBackgroundDownloadThreads(size_t threads_num)
|
||||
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_)
|
||||
: key_metadata(key_metadata_)
|
||||
, lock(key_metadata->guard.lock())
|
||||
@ -778,8 +842,8 @@ LockedKey::~LockedKey()
|
||||
/// See comment near cleanupThreadFunc() for more details.
|
||||
|
||||
key_metadata->key_state = KeyMetadata::KeyState::REMOVING;
|
||||
LOG_DEBUG(key_metadata->log, "Submitting key {} for removal", getKey());
|
||||
key_metadata->cleanup_queue->add(getKey());
|
||||
LOG_DEBUG(key_metadata->logger(), "Submitting key {} for removal", getKey());
|
||||
key_metadata->addToCleanupQueue();
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
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);
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
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);
|
||||
|
||||
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 | 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)
|
||||
{
|
||||
#ifdef ABORT_ON_LOGICAL_ERROR
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path);
|
||||
#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());
|
||||
#endif
|
||||
}
|
||||
@ -942,7 +1012,7 @@ bool LockedKey::addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &
|
||||
auto it = key_metadata->find(offset);
|
||||
if (it == key_metadata->end())
|
||||
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
|
||||
@ -1042,7 +1112,7 @@ std::vector<FileSegment::Info> LockedKey::sync()
|
||||
if (!fs::exists(path))
|
||||
{
|
||||
LOG_WARNING(
|
||||
key_metadata->log,
|
||||
key_metadata->logger(),
|
||||
"File segment has DOWNLOADED state, but file does not exist ({})",
|
||||
file_segment->getInfoForLog());
|
||||
|
||||
@ -1061,7 +1131,7 @@ std::vector<FileSegment::Info> LockedKey::sync()
|
||||
}
|
||||
|
||||
LOG_WARNING(
|
||||
key_metadata->log,
|
||||
key_metadata->logger(),
|
||||
"File segment has unexpected size. Having {}, expected {} ({})",
|
||||
actual_size, expected_size, file_segment->getInfoForLog());
|
||||
|
||||
|
@ -15,7 +15,9 @@ class CleanupQueue;
|
||||
using CleanupQueuePtr = std::shared_ptr<CleanupQueue>;
|
||||
class DownloadQueue;
|
||||
using DownloadQueuePtr = std::shared_ptr<DownloadQueue>;
|
||||
|
||||
using FileSegmentsHolderPtr = std::unique_ptr<FileSegmentsHolder>;
|
||||
class CacheMetadata;
|
||||
|
||||
|
||||
struct FileSegmentMetadata : private boost::noncopyable
|
||||
@ -43,17 +45,18 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
|
||||
private boost::noncopyable,
|
||||
public std::enable_shared_from_this<KeyMetadata>
|
||||
{
|
||||
friend class CacheMetadata;
|
||||
friend struct LockedKey;
|
||||
|
||||
using Key = FileCacheKey;
|
||||
using iterator = iterator;
|
||||
using UserInfo = FileCacheUserInfo;
|
||||
using UserID = UserInfo::UserID;
|
||||
|
||||
KeyMetadata(
|
||||
const Key & key_,
|
||||
const std::string & key_path_,
|
||||
CleanupQueuePtr cleanup_queue_,
|
||||
DownloadQueuePtr download_queue_,
|
||||
Poco::Logger * log_,
|
||||
std::shared_mutex & key_prefix_directory_mutex_,
|
||||
const UserInfo & user_id_,
|
||||
const CacheMetadata * cache_metadata_,
|
||||
bool created_base_directory_ = false);
|
||||
|
||||
enum class KeyState
|
||||
@ -64,19 +67,22 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
|
||||
};
|
||||
|
||||
const Key key;
|
||||
const std::string key_path;
|
||||
const UserInfo user;
|
||||
|
||||
LockedKeyPtr lock();
|
||||
|
||||
/// Return nullptr if key has non-ACTIVE state.
|
||||
LockedKeyPtr tryLock();
|
||||
|
||||
LockedKeyPtr lockNoStateCheck();
|
||||
|
||||
bool createBaseDirectory();
|
||||
|
||||
std::string getPath() 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,
|
||||
/// 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.
|
||||
@ -85,41 +91,52 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
|
||||
size_t sizeUnlocked() const { return size(); }
|
||||
|
||||
private:
|
||||
const CacheMetadata * cache_metadata;
|
||||
|
||||
KeyState key_state = KeyState::ACTIVE;
|
||||
KeyGuard guard;
|
||||
const CleanupQueuePtr cleanup_queue;
|
||||
const DownloadQueuePtr download_queue;
|
||||
std::shared_mutex & key_prefix_directory_mutex;
|
||||
|
||||
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>;
|
||||
|
||||
|
||||
struct CacheMetadata
|
||||
class CacheMetadata : private boost::noncopyable
|
||||
{
|
||||
friend struct KeyMetadata;
|
||||
public:
|
||||
using Key = FileCacheKey;
|
||||
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();
|
||||
|
||||
bool isEmpty() const;
|
||||
|
||||
const String & getBaseDirectory() const { return path; }
|
||||
|
||||
String getPathForFileSegment(
|
||||
String getKeyPath(const Key & key, const UserInfo & user) const;
|
||||
|
||||
String getFileSegmentPath(
|
||||
const Key & key,
|
||||
size_t offset,
|
||||
FileSegmentKind segment_kind) const;
|
||||
FileSegmentKind segment_kind,
|
||||
const UserInfo & user) const;
|
||||
|
||||
String getPathForKey(const Key & key) const;
|
||||
static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind);
|
||||
|
||||
void iterate(IterateFunc && func);
|
||||
|
||||
bool isEmpty() const;
|
||||
void iterate(IterateFunc && func, const UserID & user_id);
|
||||
|
||||
enum class KeyNotFoundPolicy
|
||||
{
|
||||
@ -132,15 +149,17 @@ public:
|
||||
KeyMetadataPtr getKeyMetadata(
|
||||
const Key & key,
|
||||
KeyNotFoundPolicy key_not_found_policy,
|
||||
const UserInfo & user,
|
||||
bool is_initial_load = false);
|
||||
|
||||
LockedKeyPtr lockKeyMetadata(
|
||||
const Key & key,
|
||||
KeyNotFoundPolicy key_not_found_policy,
|
||||
const UserInfo & user,
|
||||
bool is_initial_load = false);
|
||||
|
||||
void removeKey(const Key & key, bool if_exists, bool if_releasable);
|
||||
void removeAllKeys(bool if_releasable);
|
||||
void removeKey(const Key & key, bool if_exists, bool if_releasable, const UserID & user_id);
|
||||
void removeAllKeys(bool if_releasable, const UserID & user_id);
|
||||
|
||||
void shutdown();
|
||||
|
||||
@ -151,12 +170,15 @@ public:
|
||||
bool isBackgroundDownloadEnabled();
|
||||
|
||||
private:
|
||||
const std::string path; /// Cache base path
|
||||
static constexpr size_t buckets_num = 1024;
|
||||
|
||||
const std::string path;
|
||||
const CleanupQueuePtr cleanup_queue;
|
||||
const DownloadQueuePtr download_queue;
|
||||
const bool write_cache_per_user_directory;
|
||||
|
||||
std::shared_mutex key_prefix_directory_mutex;
|
||||
Poco::Logger * log;
|
||||
mutable std::shared_mutex key_prefix_directory_mutex;
|
||||
|
||||
struct MetadataBucket : public std::unordered_map<FileCacheKey, KeyMetadataPtr>
|
||||
{
|
||||
@ -165,7 +187,6 @@ private:
|
||||
mutable CacheMetadataGuard guard;
|
||||
};
|
||||
|
||||
static constexpr size_t buckets_num = 1024;
|
||||
std::vector<MetadataBucket> metadata_buckets{buckets_num};
|
||||
|
||||
struct DownloadThread
|
||||
@ -173,13 +194,15 @@ private:
|
||||
std::unique_ptr<ThreadFromGlobalPool> thread;
|
||||
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;
|
||||
|
||||
static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind);
|
||||
|
||||
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 & bucket,
|
||||
MetadataBucket::iterator it,
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/Cache/QueryLimit.h>
|
||||
#include <Interpreters/Cache/Metadata.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -68,9 +69,10 @@ void FileCacheQueryLimit::QueryContext::add(
|
||||
KeyMetadataPtr key_metadata,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const FileCache::UserInfo & user,
|
||||
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);
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -44,6 +44,7 @@ public:
|
||||
KeyMetadataPtr key_metadata,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const FileCacheUserInfo & user,
|
||||
const CacheGuard::Lock &);
|
||||
|
||||
void remove(
|
||||
@ -58,6 +59,21 @@ public:
|
||||
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:
|
||||
using QueryContextMap = std::unordered_map<String, QueryContextPtr>;
|
||||
QueryContextMap query_map;
|
||||
|
@ -21,11 +21,13 @@ namespace
|
||||
SLRUFileCachePriority::SLRUFileCachePriority(
|
||||
size_t max_size_,
|
||||
size_t max_elements_,
|
||||
double size_ratio_)
|
||||
double size_ratio_,
|
||||
LRUFileCachePriority::StatePtr probationary_state_,
|
||||
LRUFileCachePriority::StatePtr protected_state_)
|
||||
: IFileCachePriority(max_size_, max_elements_)
|
||||
, size_ratio(size_ratio_)
|
||||
, protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, size_ratio)))
|
||||
, probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - 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_state_))
|
||||
{
|
||||
LOG_DEBUG(
|
||||
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);
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
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
|
||||
KeyMetadataPtr key_metadata,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const UserInfo &,
|
||||
const CacheGuard::Lock & lock,
|
||||
bool is_startup)
|
||||
{
|
||||
@ -83,13 +102,14 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
|
||||
EvictionCandidates & res,
|
||||
IFileCachePriority::IteratorPtr reservee,
|
||||
FinalizeEvictionFunc & finalize_eviction_func,
|
||||
const UserID & user_id,
|
||||
const CacheGuard::Lock & lock)
|
||||
{
|
||||
/// 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.
|
||||
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),
|
||||
@ -97,7 +117,7 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
|
||||
/// (in order to know where we need to free space).
|
||||
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.
|
||||
@ -114,13 +134,13 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
|
||||
FileCacheReserveStat downgrade_stat;
|
||||
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;
|
||||
|
||||
const size_t size_to_downgrade = downgrade_stat.stat.releasable_size;
|
||||
|
||||
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;
|
||||
|
||||
finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable
|
||||
@ -168,7 +188,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
|
||||
FileCacheReserveStat downgrade_stat;
|
||||
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
|
||||
/// (not enough releasable file segments).
|
||||
@ -191,7 +211,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
|
||||
|
||||
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,
|
||||
/// 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;
|
||||
}
|
||||
|
||||
std::vector<FileSegmentInfo> SLRUFileCachePriority::dump(const CacheGuard::Lock & lock)
|
||||
IFileCachePriority::PriorityDumpPtr SLRUFileCachePriority::dump(const CacheGuard::Lock & lock)
|
||||
{
|
||||
auto res = probationary_queue.dump(lock);
|
||||
auto part_res = protected_queue.dump(lock);
|
||||
res.insert(res.end(), part_res.begin(), part_res.end());
|
||||
auto res = dynamic_pointer_cast<LRUFileCachePriority::LRUPriorityDump>(probationary_queue.dump(lock));
|
||||
auto part_res = dynamic_pointer_cast<LRUFileCachePriority::LRUPriorityDump>(protected_queue.dump(lock));
|
||||
res->merge(*part_res);
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -264,7 +284,6 @@ SLRUFileCachePriority::SLRUIterator::SLRUIterator(
|
||||
|
||||
SLRUFileCachePriority::EntryPtr SLRUFileCachePriority::SLRUIterator::getEntry() const
|
||||
{
|
||||
chassert(entry == lru_iterator.getEntry());
|
||||
return entry;
|
||||
}
|
||||
|
||||
|
@ -14,18 +14,28 @@ class SLRUFileCachePriority : public IFileCachePriority
|
||||
public:
|
||||
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 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
|
||||
KeyMetadataPtr key_metadata,
|
||||
size_t offset,
|
||||
size_t size,
|
||||
const UserInfo & user,
|
||||
const CacheGuard::Lock &,
|
||||
bool is_startup = false) override;
|
||||
|
||||
@ -35,11 +45,12 @@ public:
|
||||
EvictionCandidates & res,
|
||||
IFileCachePriority::IteratorPtr reservee,
|
||||
FinalizeEvictionFunc & finalize_eviction_func,
|
||||
const UserID & user_id,
|
||||
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;
|
||||
|
||||
|
24
src/Interpreters/Cache/UserInfo.h
Normal file
24
src/Interpreters/Cache/UserInfo.h
Normal 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; }
|
||||
};
|
||||
|
||||
}
|
@ -19,7 +19,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
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_)
|
||||
{
|
||||
}
|
||||
@ -27,7 +27,7 @@ WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegment * file_segment_)
|
||||
WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegmentsHolderPtr segment_holder_)
|
||||
: WriteBufferFromFileDecorator(
|
||||
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"))
|
||||
, file_segment(&segment_holder_->front())
|
||||
, segment_holder(std::move(segment_holder_))
|
||||
@ -84,7 +84,7 @@ void WriteBufferToFileSegment::nextImpl()
|
||||
std::unique_ptr<ReadBuffer> WriteBufferToFileSegment::getReadBufferImpl()
|
||||
{
|
||||
finalize();
|
||||
return std::make_unique<ReadBufferFromFile>(file_segment->getPathInLocalCache());
|
||||
return std::make_unique<ReadBufferFromFile>(file_segment->getPath());
|
||||
}
|
||||
|
||||
WriteBufferToFileSegment::~WriteBufferToFileSegment()
|
||||
|
@ -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 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_cache_user TSA_GUARDED_BY(mutex);
|
||||
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.
|
||||
|
||||
@ -886,6 +887,12 @@ String Context::getFilesystemCachesPath() const
|
||||
return shared->filesystem_caches_path;
|
||||
}
|
||||
|
||||
String Context::getFilesystemCacheUser() const
|
||||
{
|
||||
SharedLockGuard lock(shared->mutex);
|
||||
return shared->filesystem_cache_user;
|
||||
}
|
||||
|
||||
Strings Context::getWarnings() const
|
||||
{
|
||||
Strings common_warnings;
|
||||
@ -993,6 +1000,12 @@ void Context::setFilesystemCachesPath(const String & 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)
|
||||
try
|
||||
{
|
||||
|
@ -529,6 +529,7 @@ public:
|
||||
String getDictionariesLibPath() const;
|
||||
String getUserScriptsPath() const;
|
||||
String getFilesystemCachesPath() const;
|
||||
String getFilesystemCacheUser() const;
|
||||
|
||||
/// A list of warnings about server configuration to place in `system.warnings` table.
|
||||
Strings getWarnings() const;
|
||||
@ -540,6 +541,7 @@ public:
|
||||
void setTempDataOnDisk(TemporaryDataOnDiskScopePtr temp_data_on_disk_);
|
||||
|
||||
void setFilesystemCachesPath(const String & path);
|
||||
void setFilesystemCacheUser(const String & user);
|
||||
|
||||
void setPath(const String & path);
|
||||
void setFlagsPath(const String & path);
|
||||
|
@ -384,22 +384,22 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
{
|
||||
auto caches = FileCacheFactory::instance().getAll();
|
||||
for (const auto & [_, cache_data] : caches)
|
||||
cache_data->cache->removeAllReleasable();
|
||||
cache_data->cache->removeAllReleasable(FileCache::getCommonUser().user_id);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name)->cache;
|
||||
if (query.key_to_drop.empty())
|
||||
{
|
||||
cache->removeAllReleasable();
|
||||
cache->removeAllReleasable(FileCache::getCommonUser().user_id);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto key = FileCacheKey::fromKeyString(query.key_to_drop);
|
||||
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
|
||||
cache->removeKey(key);
|
||||
cache->removeKey(key, FileCache::getCommonUser().user_id);
|
||||
}
|
||||
}
|
||||
break;
|
||||
@ -424,7 +424,9 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
for (const auto & file_segment : file_segments)
|
||||
{
|
||||
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(path);
|
||||
res_columns[i++]->insert(file_segment.downloaded_size);
|
||||
|
@ -105,7 +105,9 @@ FileSegmentsHolderPtr TemporaryDataOnDisk::createCacheFile(size_t max_file_size)
|
||||
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);
|
||||
|
||||
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);
|
||||
holder->back().getKeyMetadata()->createBaseDirectory();
|
||||
@ -377,7 +379,7 @@ String TemporaryFileStream::getPath() const
|
||||
if (file)
|
||||
return file->getAbsolutePath();
|
||||
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");
|
||||
}
|
||||
|
@ -348,7 +348,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
|
||||
if (!data_part_storage.isDirectory(file_name))
|
||||
{
|
||||
auto remote_path = data_part_storage.getRemotePath(file_name);
|
||||
cache.removePathIfExists(remote_path);
|
||||
cache.removePathIfExists(remote_path, FileCache::getCommonUser().user_id);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -30,6 +30,7 @@ ColumnsDescription StorageSystemFilesystemCache::getColumnsDescription()
|
||||
{"downloaded_size", std::make_shared<DataTypeUInt64>()},
|
||||
{"kind", std::make_shared<DataTypeString>()},
|
||||
{"unbound", std::make_shared<DataTypeNumber<UInt8>>()},
|
||||
{"user_id", std::make_shared<DataTypeString>()},
|
||||
{"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->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)
|
||||
|
||||
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(file_segment.key.toString());
|
||||
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(toString(file_segment.kind));
|
||||
res_columns[i++]->insert(file_segment.is_unbound);
|
||||
res_columns[i++]->insert(file_segment.user_id);
|
||||
|
||||
std::error_code ec;
|
||||
auto size = fs::file_size(path, ec);
|
||||
@ -74,7 +78,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
|
||||
res_columns[i++]->insert(size);
|
||||
else
|
||||
res_columns[i++]->insertDefault();
|
||||
});
|
||||
}, FileCache::getCommonUser().user_id);
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user