diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 76d54f9d27c..59cf8f0174c 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -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 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}; diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 8faf23ad343..b4e7701de75 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -147,8 +147,6 @@ private: FileCache::QueryContextHolderPtr query_context_holder; - bool is_persistent; - std::shared_ptr cache_log; }; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index b7727555480..33d4ed7b3d7 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -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; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 8642886d6de..8d39b6eed42 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -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; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index b1cf8226895..3e7c4d12c42 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -119,7 +119,6 @@ std::unique_ptr 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, diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index ba9fbd02d94..76f16c9d930 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -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); diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index dae4261e92c..87f249823b2 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -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; diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index e160796d9a3..8f22e44145a 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -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; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index b601c4029f3..b9e189747ac 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -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(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") { diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 9f99e941110..a93ef669898 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -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; diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 47fdb65bec4..dbe3e8ced02 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -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); diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index eeb2a02c131..cddb8fd459b 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -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; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 681c0d719e4..062f168a046 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -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; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f799bae1e10..77b9deb7846 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -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; diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index ca875ee57b2..1d2c567d16f 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -24,7 +24,6 @@ static Block getSampleBlock() ColumnWithTypeAndName{std::make_shared(), "current_size"}, ColumnWithTypeAndName{std::make_shared(), "current_elements"}, ColumnWithTypeAndName{std::make_shared(), "path"}, - ColumnWithTypeAndName{std::make_shared>(), "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(); diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index 8e9ad2ac501..e03fd9ca081 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -26,7 +26,6 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes() {"cache_hits", std::make_shared()}, {"references", std::make_shared()}, {"downloaded_size", std::make_shared()}, - {"persistent", std::make_shared>()}, {"kind", std::make_shared()}, {"unbound", std::make_shared>()}, }; @@ -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()); } } } diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index dee03307177..deee71bd812 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -57,7 +57,6 @@ s3_cache/ 128Mi 1 - 0 100 @@ -65,7 +64,6 @@ s3_disk_2 s3_cache_2/ 128Mi - 0 100Mi 100 @@ -77,7 +75,6 @@ 22548578304 1 1 - 0 100 @@ -87,7 +84,6 @@ 128Mi 1 1 - 0 100 @@ -95,7 +91,6 @@ s3_disk_5 s3_cache_5/ 128Mi - 0 100 @@ -103,7 +98,6 @@ s3_disk_6 s3_cache_6/ 128Mi - 0 1 100 100 @@ -113,7 +107,6 @@ s3_disk_6 s3_cache_small/ 1000 - 1 100 @@ -122,7 +115,6 @@ s3_cache_small_segment_size/ 128Mi 10Ki - 0 1 100 @@ -146,7 +138,6 @@ local_cache/ 22548578304 1 - 0 100 @@ -154,7 +145,6 @@ local_disk local_cache_2/ 22548578304 - 0 100 @@ -164,7 +154,6 @@ 22548578304 1 1 - 0 100 @@ -173,7 +162,6 @@ s3_cache_5 s3_cache_multi/ 22548578304 - 0 100 @@ -181,7 +169,6 @@ s3_cache_multi s3_cache_multi_2/ 22548578304 - 0 100 diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index b139a57175e..303fff66166 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -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, - "1", - "0", - ) - - 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, - "0", - "1", - ) - - 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] diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index d8a2ffab1fa..2e6df4a6a0b 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -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/ diff --git a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.reference b/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.reference deleted file mode 100644 index e77afc98007..00000000000 --- a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.reference +++ /dev/null @@ -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; diff --git a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.sql b/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.sql deleted file mode 100644 index 6486840602e..00000000000 --- a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.sql +++ /dev/null @@ -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;