mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
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:
commit
c0889e5832
@ -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));
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
|
@ -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>;
|
||||
|
@ -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>
|
||||
|
@ -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;
|
@ -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;
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user