mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #51253 from kssenii/delete-non-evictable-feature
Delete feature do_not_evict_index_and_mark_files in fs cache
This commit is contained in:
commit
c66ff09627
@ -69,7 +69,6 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
|
||||
, allow_seeks_after_first_read(allow_seeks_after_first_read_)
|
||||
, use_external_buffer(use_external_buffer_)
|
||||
, query_context_holder(cache_->getQueryContextHolder(query_id, settings_))
|
||||
, is_persistent(settings_.is_file_cache_persistent)
|
||||
, cache_log(cache_log_)
|
||||
{
|
||||
}
|
||||
@ -125,7 +124,7 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size)
|
||||
}
|
||||
else
|
||||
{
|
||||
CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular);
|
||||
CreateFileSegmentSettings create_settings(FileSegmentKind::Regular);
|
||||
file_segments = cache->getOrSet(cache_key, offset, size, file_size.value(), create_settings);
|
||||
}
|
||||
|
||||
@ -149,8 +148,6 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::CachedReadBufferCreateBufferMicroseconds);
|
||||
|
||||
/// Use is_persistent flag from in-memory state of the filesegment,
|
||||
/// because it is consistent with what is written on disk.
|
||||
auto path = file_segment.getPathInLocalCache();
|
||||
|
||||
ReadSettings local_read_settings{settings};
|
||||
|
@ -147,8 +147,6 @@ private:
|
||||
|
||||
FileCache::QueryContextHolderPtr query_context_holder;
|
||||
|
||||
bool is_persistent;
|
||||
|
||||
std::shared_ptr<FilesystemCacheLog> cache_log;
|
||||
};
|
||||
|
||||
|
@ -194,7 +194,6 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
|
||||
FileCachePtr cache_,
|
||||
const String & source_path_,
|
||||
const FileCache::Key & key_,
|
||||
bool is_persistent_cache_file_,
|
||||
const String & query_id_,
|
||||
const WriteSettings & settings_)
|
||||
: WriteBufferFromFileDecorator(std::move(impl_))
|
||||
@ -202,7 +201,6 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
|
||||
, cache(cache_)
|
||||
, source_path(source_path_)
|
||||
, key(key_)
|
||||
, is_persistent_cache_file(is_persistent_cache_file_)
|
||||
, query_id(query_id_)
|
||||
, enable_cache_log(!query_id_.empty() && settings_.enable_filesystem_cache_log)
|
||||
, throw_on_error_from_cache(settings_.throw_on_error_from_cache)
|
||||
@ -255,8 +253,7 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size, bool t
|
||||
|
||||
try
|
||||
{
|
||||
auto segment_kind = is_persistent_cache_file ? FileSegmentKind::Persistent : FileSegmentKind::Regular;
|
||||
if (!cache_writer->write(data, size, current_download_offset, segment_kind))
|
||||
if (!cache_writer->write(data, size, current_download_offset, FileSegmentKind::Regular))
|
||||
{
|
||||
LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted");
|
||||
return;
|
||||
|
@ -72,7 +72,6 @@ public:
|
||||
FileCachePtr cache_,
|
||||
const String & source_path_,
|
||||
const FileCache::Key & key_,
|
||||
bool is_persistent_cache_file_,
|
||||
const String & query_id_,
|
||||
const WriteSettings & settings_);
|
||||
|
||||
@ -89,7 +88,6 @@ private:
|
||||
String source_path;
|
||||
FileCache::Key key;
|
||||
|
||||
bool is_persistent_cache_file;
|
||||
size_t current_download_offset = 0;
|
||||
const String query_id;
|
||||
|
||||
|
@ -119,7 +119,6 @@ std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// N
|
||||
cache,
|
||||
implementation_buffer->getFileName(),
|
||||
key,
|
||||
modified_write_settings.is_file_cache_persistent,
|
||||
CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "",
|
||||
modified_write_settings);
|
||||
}
|
||||
@ -164,20 +163,6 @@ void CachedObjectStorage::removeObjectsIfExist(const StoredObjects & objects)
|
||||
object_storage->removeObjectsIfExist(objects);
|
||||
}
|
||||
|
||||
ReadSettings CachedObjectStorage::getAdjustedSettingsFromMetadataFile(const ReadSettings & settings, const std::string & path) const
|
||||
{
|
||||
ReadSettings new_settings{settings};
|
||||
new_settings.is_file_cache_persistent = isFileWithPersistentCache(path) && cache_settings.do_not_evict_index_and_mark_files;
|
||||
return new_settings;
|
||||
}
|
||||
|
||||
WriteSettings CachedObjectStorage::getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const
|
||||
{
|
||||
WriteSettings new_settings{settings};
|
||||
new_settings.is_file_cache_persistent = isFileWithPersistentCache(path) && cache_settings.do_not_evict_index_and_mark_files;
|
||||
return new_settings;
|
||||
}
|
||||
|
||||
void CachedObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT
|
||||
const StoredObject & object_from,
|
||||
const StoredObject & object_to,
|
||||
|
@ -108,10 +108,6 @@ public:
|
||||
|
||||
bool supportParallelWrite() const override { return object_storage->supportParallelWrite(); }
|
||||
|
||||
ReadSettings getAdjustedSettingsFromMetadataFile(const ReadSettings & settings, const std::string & path) const override;
|
||||
|
||||
WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override;
|
||||
|
||||
const FileCacheSettings & getCacheSettings() const { return cache_settings; }
|
||||
|
||||
static bool canUseReadThroughCache(const ReadSettings & settings);
|
||||
|
@ -98,7 +98,6 @@ struct ReadSettings
|
||||
bool enable_filesystem_cache = true;
|
||||
bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false;
|
||||
bool enable_filesystem_cache_log = false;
|
||||
bool is_file_cache_persistent = false; /// Some files can be made non-evictable.
|
||||
/// Don't populate cache when the read is not part of query execution (e.g. background thread).
|
||||
bool avoid_readthrough_cache_outside_query_context = true;
|
||||
|
||||
|
@ -19,7 +19,6 @@ struct WriteSettings
|
||||
/// Filesystem cache settings
|
||||
bool enable_filesystem_cache_on_write_operations = false;
|
||||
bool enable_filesystem_cache_log = false;
|
||||
bool is_file_cache_persistent = false;
|
||||
bool throw_on_error_from_cache = false;
|
||||
|
||||
bool s3_allow_parallel_part_upload = true;
|
||||
|
@ -52,7 +52,6 @@ namespace ErrorCodes
|
||||
|
||||
FileCache::FileCache(const FileCacheSettings & settings)
|
||||
: max_file_segment_size(settings.max_file_segment_size)
|
||||
, allow_persistent_files(settings.do_not_evict_index_and_mark_files)
|
||||
, bypass_cache_threshold(settings.enable_bypass_cache_with_threashold ? settings.bypass_cache_threashold : 0)
|
||||
, delayed_cleanup_interval_ms(settings.delayed_cleanup_interval_ms)
|
||||
, log(&Poco::Logger::get("FileCache"))
|
||||
@ -642,10 +641,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size)
|
||||
{
|
||||
chassert(segment_metadata->file_segment->assertCorrectness());
|
||||
|
||||
const bool is_persistent = allow_persistent_files && segment_metadata->file_segment->isPersistent();
|
||||
const bool releasable = segment_metadata->releasable() && !is_persistent;
|
||||
|
||||
if (releasable)
|
||||
if (segment_metadata->releasable())
|
||||
{
|
||||
auto segment = segment_metadata->file_segment;
|
||||
if (segment->state() == FileSegment::State::DOWNLOADED)
|
||||
@ -833,10 +829,6 @@ void FileCache::removeAllReleasable()
|
||||
{
|
||||
assertInitialized();
|
||||
|
||||
/// Only releasable file segments are evicted.
|
||||
/// `remove_persistent_files` defines whether non-evictable by some criteria files
|
||||
/// (they do not comply with the cache eviction policy) should also be removed.
|
||||
|
||||
auto lock = lockCache();
|
||||
|
||||
main_priority->iterate([&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata)
|
||||
@ -939,7 +931,9 @@ void FileCache::loadMetadata()
|
||||
parsed = tryParse<UInt64>(offset, offset_with_suffix.substr(0, delim_pos));
|
||||
if (offset_with_suffix.substr(delim_pos+1) == "persistent")
|
||||
{
|
||||
segment_kind = FileSegmentKind::Persistent;
|
||||
/// For compatibility. Persistent files are no longer supported.
|
||||
fs::remove(offset_it->path());
|
||||
continue;
|
||||
}
|
||||
if (offset_with_suffix.substr(delim_pos+1) == "temporary")
|
||||
{
|
||||
|
@ -134,7 +134,6 @@ private:
|
||||
using KeyAndOffset = FileCacheKeyAndOffset;
|
||||
|
||||
const size_t max_file_segment_size;
|
||||
const bool allow_persistent_files;
|
||||
const size_t bypass_cache_threshold = 0;
|
||||
const size_t delayed_cleanup_interval_ms;
|
||||
|
||||
|
@ -47,8 +47,6 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration &
|
||||
else
|
||||
bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD;
|
||||
|
||||
do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", false);
|
||||
|
||||
boundary_alignment = config.getUInt64(config_prefix + ".boundary_alignment", DBMS_DEFAULT_BUFFER_SIZE);
|
||||
|
||||
delayed_cleanup_interval_ms = config.getUInt64(config_prefix + ".delayed_cleanup_interval_ms", FILECACHE_DELAYED_CLEANUP_INTERVAL_MS);
|
||||
|
@ -22,8 +22,6 @@ struct FileCacheSettings
|
||||
size_t cache_hits_threshold = FILECACHE_DEFAULT_HITS_THRESHOLD;
|
||||
bool enable_filesystem_query_cache_limit = false;
|
||||
|
||||
bool do_not_evict_index_and_mark_files = true;
|
||||
|
||||
bool enable_bypass_cache_with_threashold = false;
|
||||
size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD;
|
||||
size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS;
|
||||
|
@ -37,11 +37,6 @@ enum class FileSegmentKind
|
||||
*/
|
||||
Regular,
|
||||
|
||||
/* `Persistent` file segment can't be evicted from cache,
|
||||
* it should be removed manually.
|
||||
*/
|
||||
Persistent,
|
||||
|
||||
/* `Temporary` file segment is removed right after releasing.
|
||||
* Also corresponding files are removed during cache loading (if any).
|
||||
*/
|
||||
@ -155,8 +150,6 @@ public:
|
||||
|
||||
FileSegmentKind getKind() const { return segment_kind; }
|
||||
|
||||
bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; }
|
||||
|
||||
bool isUnbound() const { return is_unbound; }
|
||||
|
||||
String getPathInLocalCache() const;
|
||||
|
@ -143,9 +143,6 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s
|
||||
String file_suffix;
|
||||
switch (segment_kind)
|
||||
{
|
||||
case FileSegmentKind::Persistent:
|
||||
file_suffix = "_persistent";
|
||||
break;
|
||||
case FileSegmentKind::Temporary:
|
||||
file_suffix = "_temporary";
|
||||
break;
|
||||
|
@ -24,7 +24,6 @@ static Block getSampleBlock()
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "current_size"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeUInt64>(), "current_elements"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeString>(), "path"},
|
||||
ColumnWithTypeAndName{std::make_shared<DataTypeNumber<UInt8>>(), "do_not_evict_index_and_mark_files"},
|
||||
};
|
||||
return Block(columns);
|
||||
}
|
||||
@ -49,7 +48,6 @@ BlockIO InterpreterDescribeCacheQuery::execute()
|
||||
res_columns[5]->insert(cache->getUsedCacheSize());
|
||||
res_columns[6]->insert(cache->getFileSegmentsNum());
|
||||
res_columns[7]->insert(cache->getBasePath());
|
||||
res_columns[8]->insert(settings.do_not_evict_index_and_mark_files);
|
||||
|
||||
BlockIO res;
|
||||
size_t num_rows = res_columns[0]->size();
|
||||
|
@ -26,7 +26,6 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes()
|
||||
{"cache_hits", std::make_shared<DataTypeUInt64>()},
|
||||
{"references", std::make_shared<DataTypeUInt64>()},
|
||||
{"downloaded_size", std::make_shared<DataTypeUInt64>()},
|
||||
{"persistent", std::make_shared<DataTypeNumber<UInt8>>()},
|
||||
{"kind", std::make_shared<DataTypeString>()},
|
||||
{"unbound", std::make_shared<DataTypeNumber<UInt8>>()},
|
||||
};
|
||||
@ -48,25 +47,25 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
|
||||
|
||||
for (const auto & file_segment : *file_segments)
|
||||
{
|
||||
res_columns[0]->insert(cache_name);
|
||||
res_columns[1]->insert(cache->getBasePath());
|
||||
size_t i = 0;
|
||||
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
|
||||
/// (because file_segments in getSnapshot doesn't have `cache` field set)
|
||||
res_columns[2]->insert(cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind()));
|
||||
res_columns[3]->insert(file_segment->key().toString());
|
||||
res_columns[i++]->insert(cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind()));
|
||||
res_columns[i++]->insert(file_segment->key().toString());
|
||||
|
||||
const auto & range = file_segment->range();
|
||||
res_columns[4]->insert(range.left);
|
||||
res_columns[5]->insert(range.right);
|
||||
res_columns[6]->insert(range.size());
|
||||
res_columns[7]->insert(FileSegment::stateToString(file_segment->state()));
|
||||
res_columns[8]->insert(file_segment->getHitsCount());
|
||||
res_columns[9]->insert(file_segment->getRefCount());
|
||||
res_columns[10]->insert(file_segment->getDownloadedSize(false));
|
||||
res_columns[11]->insert(file_segment->isPersistent());
|
||||
res_columns[12]->insert(toString(file_segment->getKind()));
|
||||
res_columns[13]->insert(file_segment->isUnbound());
|
||||
res_columns[i++]->insert(range.left);
|
||||
res_columns[i++]->insert(range.right);
|
||||
res_columns[i++]->insert(range.size());
|
||||
res_columns[i++]->insert(FileSegment::stateToString(file_segment->state()));
|
||||
res_columns[i++]->insert(file_segment->getHitsCount());
|
||||
res_columns[i++]->insert(file_segment->getRefCount());
|
||||
res_columns[i++]->insert(file_segment->getDownloadedSize(false));
|
||||
res_columns[i++]->insert(toString(file_segment->getKind()));
|
||||
res_columns[i++]->insert(file_segment->isUnbound());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -57,7 +57,6 @@
|
||||
<path>s3_cache/</path>
|
||||
<max_size>128Mi</max_size>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache>
|
||||
<s3_cache_2>
|
||||
@ -65,7 +64,6 @@
|
||||
<disk>s3_disk_2</disk>
|
||||
<path>s3_cache_2/</path>
|
||||
<max_size>128Mi</max_size>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<max_file_segment_size>100Mi</max_file_segment_size>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_2>
|
||||
@ -77,7 +75,6 @@
|
||||
<data_cache_max_size>22548578304</data_cache_max_size>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<enable_cache_hits_threshold>1</enable_cache_hits_threshold>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_3>
|
||||
<s3_cache_4>
|
||||
@ -87,7 +84,6 @@
|
||||
<max_size>128Mi</max_size>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<enable_filesystem_query_cache_limit>1</enable_filesystem_query_cache_limit>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_4>
|
||||
<s3_cache_5>
|
||||
@ -95,7 +91,6 @@
|
||||
<disk>s3_disk_5</disk>
|
||||
<path>s3_cache_5/</path>
|
||||
<max_size>128Mi</max_size>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_5>
|
||||
<s3_cache_6>
|
||||
@ -103,7 +98,6 @@
|
||||
<disk>s3_disk_6</disk>
|
||||
<path>s3_cache_6/</path>
|
||||
<max_size>128Mi</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>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
@ -113,7 +107,6 @@
|
||||
<disk>s3_disk_6</disk>
|
||||
<path>s3_cache_small/</path>
|
||||
<max_size>1000</max_size>
|
||||
<do_not_evict_index_and_mark_files>1</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_small>
|
||||
<s3_cache_small_segment_size>
|
||||
@ -122,7 +115,6 @@
|
||||
<path>s3_cache_small_segment_size/</path>
|
||||
<max_size>128Mi</max_size>
|
||||
<max_file_segment_size>10Ki</max_file_segment_size>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_small_segment_size>
|
||||
@ -146,7 +138,6 @@
|
||||
<path>local_cache/</path>
|
||||
<max_size>22548578304</max_size>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</local_cache>
|
||||
<local_cache_2>
|
||||
@ -154,7 +145,6 @@
|
||||
<disk>local_disk</disk>
|
||||
<path>local_cache_2/</path>
|
||||
<max_size>22548578304</max_size>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</local_cache_2>
|
||||
<local_cache_3>
|
||||
@ -164,7 +154,6 @@
|
||||
<max_size>22548578304</max_size>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<enable_cache_hits_threshold>1</enable_cache_hits_threshold>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</local_cache_3>
|
||||
<!-- multi layer cache -->
|
||||
@ -173,7 +162,6 @@
|
||||
<disk>s3_cache_5</disk>
|
||||
<path>s3_cache_multi/</path>
|
||||
<max_size>22548578304</max_size>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_multi>
|
||||
<s3_cache_multi_2>
|
||||
@ -181,7 +169,6 @@
|
||||
<disk>s3_cache_multi</disk>
|
||||
<path>s3_cache_multi_2/</path>
|
||||
<max_size>22548578304</max_size>
|
||||
<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>
|
||||
<delayed_cleanup_interval_ms>100</delayed_cleanup_interval_ms>
|
||||
</s3_cache_multi_2>
|
||||
</disks>
|
||||
|
@ -740,84 +740,6 @@ def test_cache_with_full_disk_space(cluster, node_name):
|
||||
check_no_objects_after_drop(cluster, node_name=node_name)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_cache_setting_compatibility(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
|
||||
node.query("DROP TABLE IF EXISTS s3_test SYNC")
|
||||
|
||||
node.query(
|
||||
"CREATE TABLE s3_test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_r', compress_marks=false, compress_primary_key=false;"
|
||||
)
|
||||
node.query(
|
||||
"INSERT INTO s3_test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 500"
|
||||
)
|
||||
|
||||
result = node.query("SYSTEM DROP FILESYSTEM CACHE")
|
||||
|
||||
result = node.query(
|
||||
"SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'"
|
||||
)
|
||||
assert int(result) == 0
|
||||
|
||||
node.query("SELECT * FROM s3_test")
|
||||
|
||||
result = node.query(
|
||||
"SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'"
|
||||
)
|
||||
assert int(result) > 0
|
||||
|
||||
config_path = os.path.join(
|
||||
SCRIPT_DIR,
|
||||
f"./{cluster.instances_dir_name}/node/configs/config.d/storage_conf.xml",
|
||||
)
|
||||
|
||||
replace_config(
|
||||
config_path,
|
||||
"<do_not_evict_index_and_mark_files>1</do_not_evict_index_and_mark_files>",
|
||||
"<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>",
|
||||
)
|
||||
|
||||
result = node.query("DESCRIBE FILESYSTEM CACHE 's3_cache_r'")
|
||||
assert result.strip().endswith("1")
|
||||
|
||||
node.restart_clickhouse()
|
||||
|
||||
result = node.query("DESCRIBE FILESYSTEM CACHE 's3_cache_r'")
|
||||
assert result.strip().endswith("0")
|
||||
|
||||
result = node.query(
|
||||
"SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'"
|
||||
)
|
||||
assert int(result) > 0
|
||||
|
||||
node.query("SELECT * FROM s3_test FORMAT Null")
|
||||
|
||||
assert not node.contains_in_log("No such file or directory: Cache info:")
|
||||
|
||||
replace_config(
|
||||
config_path,
|
||||
"<do_not_evict_index_and_mark_files>0</do_not_evict_index_and_mark_files>",
|
||||
"<do_not_evict_index_and_mark_files>1</do_not_evict_index_and_mark_files>",
|
||||
)
|
||||
|
||||
result = node.query(
|
||||
"SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'"
|
||||
)
|
||||
assert int(result) > 0
|
||||
|
||||
node.restart_clickhouse()
|
||||
|
||||
result = node.query("DESCRIBE FILESYSTEM CACHE 's3_cache_r'")
|
||||
assert result.strip().endswith("1")
|
||||
|
||||
node.query("SELECT * FROM s3_test FORMAT Null")
|
||||
|
||||
assert not node.contains_in_log("No such file or directory: Cache info:")
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_merge_canceled_by_drop(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
|
@ -1,2 +1,2 @@
|
||||
134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0
|
||||
134217728 10000000 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0
|
||||
134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/
|
||||
134217728 10000000 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/
|
||||
|
@ -1,79 +0,0 @@
|
||||
-- { echo }
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
DROP TABLE IF EXISTS nopers;
|
||||
CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES nopers;
|
||||
INSERT INTO nopers SELECT number, toString(number) FROM numbers(10);
|
||||
SELECT * FROM nopers FORMAT Null;
|
||||
SELECT sum(size) FROM system.filesystem_cache;
|
||||
195
|
||||
SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
data.bin 0 114
|
||||
data.mrk3 0 80
|
||||
format_version.txt 0 1
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT sum(size) FROM system.filesystem_cache;
|
||||
1021
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
5
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
5
|
||||
SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
data.bin 0 114
|
||||
data.bin 0 746
|
||||
data.mrk3 0 80
|
||||
data.mrk3 0_persistent 80
|
||||
format_version.txt 0 1
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test2;
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100000);
|
||||
SELECT * FROM test2 FORMAT Null;
|
||||
SELECT sum(size) FROM system.filesystem_cache;
|
||||
795
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
5
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
5
|
||||
SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
data.bin 0 114
|
||||
data.mrk3 0 80
|
||||
data.mrk3 0_persistent 80
|
||||
data.mrk3 0_persistent 520
|
||||
format_version.txt 0 1
|
||||
DROP TABLE test;
|
||||
DROP TABLE test2;
|
||||
DROP TABLE nopers;
|
@ -1,76 +0,0 @@
|
||||
-- Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings
|
||||
|
||||
-- { echo }
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
|
||||
DROP TABLE IF EXISTS nopers;
|
||||
CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES nopers;
|
||||
|
||||
INSERT INTO nopers SELECT number, toString(number) FROM numbers(10);
|
||||
SELECT * FROM nopers FORMAT Null;
|
||||
SELECT sum(size) FROM system.filesystem_cache;
|
||||
|
||||
SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test;
|
||||
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
|
||||
SELECT sum(size) FROM system.filesystem_cache;
|
||||
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false;
|
||||
SYSTEM STOP MERGES test2;
|
||||
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100000);
|
||||
SELECT * FROM test2 FORMAT Null;
|
||||
|
||||
SELECT sum(size) FROM system.filesystem_cache;
|
||||
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
ORDER BY file, cache, size;
|
||||
|
||||
DROP TABLE test;
|
||||
DROP TABLE test2;
|
||||
DROP TABLE nopers;
|
Loading…
Reference in New Issue
Block a user