Merge pull request #42418 from KinderRiven/max_cache_download_limit

Supports limiting the size of cached data to avoid negative optimization
This commit is contained in:
Kseniia Sumarokova 2022-10-21 19:36:08 +02:00 committed by GitHub
commit c0889e5832
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 85 additions and 6 deletions

View File

@ -32,6 +32,8 @@ FileCache::FileCache(
, allow_persistent_files(cache_settings_.do_not_evict_index_and_mark_files)
, enable_cache_hits_threshold(cache_settings_.enable_cache_hits_threshold)
, enable_filesystem_query_cache_limit(cache_settings_.enable_filesystem_query_cache_limit)
, enable_bypass_cache_with_threashold(cache_settings_.enable_bypass_cache_with_threashold)
, bypass_cache_threashold(cache_settings_.bypass_cache_threashold)
, log(&Poco::Logger::get("FileCache"))
, main_priority(std::make_unique<LRUFileCachePriority>())
, stash_priority(std::make_unique<LRUFileCachePriority>())
@ -185,6 +187,20 @@ FileSegments FileCache::getImpl(
/// Given range = [left, right] and non-overlapping ordered set of file segments,
/// find list [segment1, ..., segmentN] of segments which intersect with given range.
FileSegments result;
if (enable_bypass_cache_with_threashold && (range.size() > bypass_cache_threashold))
{
auto file_segment = std::make_shared<FileSegment>(
range.left, range.size(), key, this, FileSegment::State::SKIP_CACHE, CreateFileSegmentSettings{});
{
std::unique_lock segment_lock(file_segment->mutex);
file_segment->detachAssumeStateFinalized(segment_lock);
}
result.emplace_back(file_segment);
return result;
}
auto it = files.find(key);
if (it == files.end())
return {};
@ -197,7 +213,6 @@ FileSegments FileCache::getImpl(
return {};
}
FileSegments result;
auto segment_it = file_segments.lower_bound(range.left);
if (segment_it == file_segments.end())
{
@ -392,7 +407,6 @@ FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t si
#endif
FileSegment::Range range(offset, offset + size - 1);
/// Get all segments which intersect with the given range.
auto file_segments = getImpl(key, range, cache_lock);
@ -404,7 +418,6 @@ FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t si
{
fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, settings, cache_lock);
}
assert(!file_segments.empty());
return FileSegmentsHolder(std::move(file_segments));
}

View File

@ -140,6 +140,9 @@ private:
const size_t enable_cache_hits_threshold;
const bool enable_filesystem_query_cache_limit;
const bool enable_bypass_cache_with_threashold;
const size_t bypass_cache_threashold;
mutable std::mutex mutex;
Poco::Logger * log;

View File

@ -35,6 +35,13 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration &
enable_filesystem_query_cache_limit = config.getUInt64(config_prefix + ".enable_filesystem_query_cache_limit", false);
enable_cache_hits_threshold = config.getUInt64(config_prefix + ".enable_cache_hits_threshold", REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD);
enable_bypass_cache_with_threashold = config.getUInt64(config_prefix + ".enable_bypass_cache_with_threashold", false);
if (config.has(config_prefix + ".bypass_cache_threashold"))
bypass_cache_threashold = parseWithSizeSuffix<uint64_t>(config.getString(config_prefix + ".bypass_cache_threashold"));
else
bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD;
do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", false);
}

View File

@ -20,6 +20,9 @@ struct FileCacheSettings
bool do_not_evict_index_and_mark_files = true;
bool enable_bypass_cache_with_threashold = false;
size_t bypass_cache_threashold = REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD;
void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
};

View File

@ -7,6 +7,7 @@ namespace DB
static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 100 * 1024 * 1024;
static constexpr int REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS = 1024 * 1024;
static constexpr int REMOTE_FS_OBJECTS_CACHE_ENABLE_HITS_THRESHOLD = 0;
static constexpr size_t REMOTE_FS_OBJECTS_CACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024;;
class FileCache;
using FileCachePtr = std::shared_ptr<FileCache>;

View File

@ -93,6 +93,15 @@
<max_size>22548578304</max_size>
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
</s3_cache_5>
<s3_cache_6>
<type>cache</type>
<disk>s3_disk_6</disk>
<path>s3_cache_6/</path>
<max_size>22548578304</max_size>
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
<enable_bypass_cache_with_threashold>1</enable_bypass_cache_with_threashold>
<bypass_cache_threashold>100</bypass_cache_threashold>
</s3_cache_6>
<s3_cache_small>
<type>cache</type>
<disk>s3_disk_6</disk>
@ -183,6 +192,13 @@
</main>
</volumes>
</s3_cache_4>
<s3_cache_6>
<volumes>
<main>
<disk>s3_cache_6</disk>
</main>
</volumes>
</s3_cache_6>
<s3_cache_multi>
<volumes>
<main>

View File

@ -0,0 +1,16 @@
-- { echo }
SYSTEM DROP FILESYSTEM CACHE;
SET enable_filesystem_cache_on_write_operations=0;
DROP TABLE IF EXISTS test;
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760;
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
SELECT * FROM test FORMAT Null;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size;
0 79 80
SYSTEM DROP FILESYSTEM CACHE;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;
SELECT * FROM test FORMAT Null;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;
SYSTEM DROP FILESYSTEM CACHE;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;

View File

@ -0,0 +1,19 @@
-- Tags: no-parallel, no-fasttest, no-s3-storage, no-random-settings
-- { echo }
SYSTEM DROP FILESYSTEM CACHE;
SET enable_filesystem_cache_on_write_operations=0;
DROP TABLE IF EXISTS test;
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760;
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
SELECT * FROM test FORMAT Null;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size;
SYSTEM DROP FILESYSTEM CACHE;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;
SELECT * FROM test FORMAT Null;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;
SYSTEM DROP FILESYSTEM CACHE;
SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache;

View File

@ -1,12 +1,13 @@
cached_azure
s3_cache_2
s3_cache
s3_cache_3
s3_cache_multi
s3_cache_4
s3_cache_5
local_cache
s3_cache_6
s3_cache_small
local_cache_2
local_cache_3
s3_cache_multi
s3_cache_3
s3_cache
s3_cache_multi_2