From 0fc92fe2aaf615b49eed9fbaf688422dbf6c589c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 30 Mar 2022 13:47:44 +0200 Subject: [PATCH] Get rid of all "remote" mentions --- docker/test/stateful/run.sh | 2 +- src/Common/FileCache.cpp | 1 + src/Common/FileCache.h | 1 - src/Core/Settings.h | 7 +-- src/Disks/DiskCacheWrapper.cpp | 6 +-- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 6 +-- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 4 +- src/IO/ReadSettings.h | 6 +-- src/IO/WriteSettings.h | 2 +- src/Interpreters/Context.cpp | 7 +-- src/Interpreters/InterpreterSystemQuery.cpp | 8 ++-- src/Parsers/ASTSystemQuery.h | 4 +- .../System/StorageSystemFilesystemCache.h | 2 +- tests/config/users.d/s3_cache.xml | 8 ++++ ...605_adaptive_granularity_block_borders.sql | 2 +- .../01641_memory_tracking_insert_optimize.sql | 2 +- .../0_stateless/01926_order_by_desc_limit.sql | 2 +- .../0_stateless/02226_s3_with_cache.sql | 6 +-- ...0_system_remote_filesystem_cache.reference | 10 ++-- .../02240_system_remote_filesystem_cache.sql | 10 ++-- ...emote_filesystem_cache_on_insert.reference | 46 +++++++++---------- ...2241_remote_filesystem_cache_on_insert.sql | 46 +++++++++---------- 23 files changed, 100 insertions(+), 90 deletions(-) create mode 100644 tests/config/users.d/s3_cache.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 77dc61e6cd0..e91acaa0b2a 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -96,7 +96,7 @@ else clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" - clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits" + clickhouse-client --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0" fi clickhouse-client --query "SHOW TABLES FROM test" diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index f0d2bba33d4..0eb53f71bc4 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -720,6 +720,7 @@ std::vector LRUFileCache::tryGetCachePaths(const Key & key) std::lock_guard cache_lock(mutex); std::vector cache_paths; + const auto & cells_by_offset = files[key]; for (const auto & [offset, cell] : cells_by_offset) diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 3a444b1c201..e8280fba08a 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -43,7 +43,6 @@ public: virtual void tryRemoveAll() = 0; - /// If cache can be used as read only. (For merges, for example). static bool shouldBypassCache(); /// Cache capacity in bytes. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 86dc16f6220..8891a49a7de 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -555,9 +555,10 @@ class IColumn; \ M(UInt64, remote_fs_read_max_backoff_ms, 10000, "Max wait time when trying to read data for remote disk", 0) \ M(UInt64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \ - M(Bool, remote_fs_enable_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must me done via disk config), but allows to bypass cache for some queries if intended", 0) \ - M(UInt64, remote_fs_cache_max_wait_sec, 5, "Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded", 0) \ - M(Bool, remote_fs_cache_on_write_operations, true, "Write into cache on INSERT query To actually work this setting requires be added to disk config too", 0) \ + M(Bool, enable_filesystem_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must me done via disk config), but allows to bypass cache for some queries if intended", 0) \ + M(UInt64, filesystem_cache_max_wait_sec, 5, "Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded", 0) \ + M(Bool, enable_filesystem_cache_on_write_operations, false, "Write into cache on write operations. To actually work this setting requires be added to disk config too", 0) \ + M(Bool, read_from_filesystem_cache_if_exists_otherwise_bypass_cache, false, "", 0) \ \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 568fbf160c0..178caa0c496 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -150,7 +150,7 @@ DiskCacheWrapper::readFile( /// Note: enabling `threadpool` read requires to call setReadUntilEnd(). current_read_settings.remote_fs_method = RemoteFSReadMethod::read; /// Disable data cache. - current_read_settings.remote_fs_enable_cache = false; + current_read_settings.enable_filesystem_cache = false; if (metadata->status == DOWNLOADING) { @@ -169,7 +169,7 @@ DiskCacheWrapper::readFile( auto src_buffer = DiskDecorator::readFile(path, current_read_settings, read_hint, file_size); WriteSettings write_settings; - write_settings.remote_fs_cache_on_write_operations = false; + write_settings.enable_filesystem_cache_on_write_operations = false; auto dst_buffer = cache_disk->writeFile(tmp_path, settings.local_fs_buffer_size, WriteMode::Rewrite, write_settings); copyData(*src_buffer, *dst_buffer); @@ -206,7 +206,7 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode return DiskDecorator::writeFile(path, buf_size, mode, settings); WriteSettings current_settings = settings; - current_settings.remote_fs_cache_on_write_operations = false; + current_settings.enable_filesystem_cache_on_write_operations = false; LOG_TEST(log, "Write file {} to cache", backQuote(path)); diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 0bd06e44496..6aa5f71139a 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -120,12 +120,12 @@ SeekableReadBufferPtr CachedReadBufferFromRemoteFS::getReadBufferForFileSegment( auto range = file_segment->range(); /// Each wait() call has a timeout of 1 second. - size_t wait_download_max_tries = settings.remote_fs_cache_max_wait_sec; + size_t wait_download_max_tries = settings.filesystem_cache_max_wait_sec; size_t wait_download_tries = 0; auto download_state = file_segment->state(); - if (settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) + if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) { if (download_state == FileSegment::State::DOWNLOADED) { @@ -772,7 +772,7 @@ std::optional CachedReadBufferFromRemoteFS::getLastNonDownloadedOffset() void CachedReadBufferFromRemoteFS::assertCacheAllowed() const { - if (IFileCache::shouldBypassCache() && !settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache) + if (IFileCache::shouldBypassCache() && !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache used when not allowed"); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8f91804bbbe..abbcd5c8add 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -38,7 +38,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S current_path = path; auto cache = settings.remote_fs_cache; - bool with_cache = cache && settings.remote_fs_enable_cache && !IFileCache::shouldBypassCache(); + bool with_cache = cache && settings.enable_filesystem_cache && !IFileCache::shouldBypassCache(); auto remote_file_reader_creator = [=, this]() { diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index e682adb1487..07d27f67d1e 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -231,7 +231,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co if (cache) { if (IFileCache::shouldBypassCache()) - disk_read_settings.remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = true; + disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; disk_read_settings.remote_fs_cache = cache; } @@ -273,7 +273,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), remote_fs_root_path + blob_name); bool cache_on_insert = fs::path(path).extension() != ".tmp" - && write_settings.remote_fs_cache_on_write_operations + && write_settings.enable_filesystem_cache_on_write_operations && FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations; auto s3_buffer = std::make_unique( diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 936de1673b4..92346615a7a 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -77,9 +77,9 @@ struct ReadSettings size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_backoff_max_tries = 4; - bool remote_fs_enable_cache = true; - size_t remote_fs_cache_max_wait_sec = 1; - bool remote_fs_read_from_cache_if_exists_otherwise_bypass_cache = false; + bool enable_filesystem_cache = true; + size_t filesystem_cache_max_wait_sec = 1; + bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false; size_t remote_read_min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE; diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 81a6705cbab..af26452e8e6 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -5,7 +5,7 @@ namespace DB struct WriteSettings { - bool remote_fs_cache_on_write_operations = false; + bool enable_filesystem_cache_on_write_operations = false; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a3169f435e4..83236781418 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3179,8 +3179,9 @@ ReadSettings Context::getReadSettings() const res.remote_fs_read_max_backoff_ms = settings.remote_fs_read_max_backoff_ms; res.remote_fs_read_backoff_max_tries = settings.remote_fs_read_backoff_max_tries; - res.remote_fs_enable_cache = settings.remote_fs_enable_cache; - res.remote_fs_cache_max_wait_sec = settings.remote_fs_cache_max_wait_sec; + res.enable_filesystem_cache = settings.enable_filesystem_cache; + res.filesystem_cache_max_wait_sec = settings.filesystem_cache_max_wait_sec; + res.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; res.remote_read_min_bytes_for_seek = settings.remote_read_min_bytes_for_seek; @@ -3203,7 +3204,7 @@ WriteSettings Context::getWriteSettings() const { WriteSettings res; - res.remote_fs_cache_on_write_operations = settings.remote_fs_cache_on_write_operations; + res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; return res; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 5f030159b2e..353e5393a03 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -298,9 +298,9 @@ BlockIO InterpreterSystemQuery::execute() cache->reset(); break; #endif - case Type::DROP_REMOTE_FILESYSTEM_CACHE: + case Type::DROP_FILESYSTEM_CACHE: { - if (query.remote_filesystem_cache_path.empty()) + if (query.filesystem_cache_path.empty()) { auto caches = FileCacheFactory::instance().getAll(); for (const auto & [_, cache_data] : caches) @@ -308,7 +308,7 @@ BlockIO InterpreterSystemQuery::execute() } else { - auto cache = FileCacheFactory::instance().get(query.remote_filesystem_cache_path); + auto cache = FileCacheFactory::instance().get(query.filesystem_cache_path); cache->tryRemoveAll(); } break; @@ -775,7 +775,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_UNCOMPRESSED_CACHE: case Type::DROP_INDEX_MARK_CACHE: case Type::DROP_INDEX_UNCOMPRESSED_CACHE: - case Type::DROP_REMOTE_FILESYSTEM_CACHE: + case Type::DROP_FILESYSTEM_CACHE: { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); break; diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 7113698789f..600525f9abe 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -28,7 +28,7 @@ public: #if USE_EMBEDDED_COMPILER DROP_COMPILED_EXPRESSION_CACHE, #endif - DROP_REMOTE_FILESYSTEM_CACHE, + DROP_FILESYSTEM_CACHE, STOP_LISTEN_QUERIES, START_LISTEN_QUERIES, RESTART_REPLICAS, @@ -89,7 +89,7 @@ public: String volume; String disk; UInt64 seconds{}; - String remote_filesystem_cache_path; + String filesystem_cache_path; String getID(char) const override { return "SYSTEM query"; } diff --git a/src/Storages/System/StorageSystemFilesystemCache.h b/src/Storages/System/StorageSystemFilesystemCache.h index cfd938a50ce..0f0bd81e760 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.h +++ b/src/Storages/System/StorageSystemFilesystemCache.h @@ -23,7 +23,7 @@ namespace DB * remote_path * FROM system.remote_data_paths * ) AS data_paths - * INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path + * INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path * FORMAT Vertical */ diff --git a/tests/config/users.d/s3_cache.xml b/tests/config/users.d/s3_cache.xml new file mode 100644 index 00000000000..69b24ecbbc4 --- /dev/null +++ b/tests/config/users.d/s3_cache.xml @@ -0,0 +1,8 @@ + + + + 1 + 1 + + + diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 750809da338..7654be4eb29 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -22,7 +22,7 @@ OPTIMIZE TABLE adaptive_table FINAL; SELECT marks FROM system.parts WHERE table = 'adaptive_table' and database=currentDatabase() and active; -SET remote_fs_enable_cache = 0; +SET enable_filesystem_cache = 0; -- If we have computed granularity incorrectly than we will exceed this limit. SET max_memory_usage='30M'; diff --git a/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql b/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql index 7ec3153886c..36b6c97460c 100644 --- a/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql +++ b/tests/queries/0_stateless/01641_memory_tracking_insert_optimize.sql @@ -3,7 +3,7 @@ drop table if exists data_01641; -- Disable cache for s3 storage tests because it increases memory usage. -set remote_fs_enable_cache=0; +set enable_filesystem_cache=0; set remote_filesystem_read_method='read'; create table data_01641 (key Int, value String) engine=MergeTree order by (key, repeat(value, 40)) settings old_parts_lifetime=0, min_bytes_for_wide_part=0; diff --git a/tests/queries/0_stateless/01926_order_by_desc_limit.sql b/tests/queries/0_stateless/01926_order_by_desc_limit.sql index 9f65cf73252..86468b4fcd6 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS order_by_desc; -SET remote_fs_enable_cache=0; +SET enable_filesystem_cache=0; CREATE TABLE order_by_desc (u UInt32, s String) ENGINE MergeTree ORDER BY u PARTITION BY u % 100 diff --git a/tests/queries/0_stateless/02226_s3_with_cache.sql b/tests/queries/0_stateless/02226_s3_with_cache.sql index 5b0d4ff3e44..d470f2ef140 100644 --- a/tests/queries/0_stateless/02226_s3_with_cache.sql +++ b/tests/queries/0_stateless/02226_s3_with_cache.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel, no-fasttest, long SET max_memory_usage='20G'; -SET remote_fs_cache_on_write_operations = 0; +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'; @@ -43,12 +43,12 @@ SET remote_filesystem_read_method='threadpool'; SELECT * FROM test WHERE value LIKE '%abc%' ORDER BY value LIMIT 10 FORMAT Null; -SET remote_fs_cache_on_write_operations = 1; +SET enable_filesystem_cache_on_write_operations = 1; TRUNCATE TABLE test; SELECT count() FROM test; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 10000; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference index 59c4d43d8ae..20cb2329604 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.reference @@ -1,19 +1,19 @@ -- { echo } -SYSTEM DROP REMOTE FILESYSTEM CACHE; -SET remote_fs_cache_on_write_operations=0; +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'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; ./disks/s3/data_cache/ 0 0 1 ./disks/s3/data_cache/ 0 79 80 ./disks/s3/data_cache/ 0 745 746 -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; ./disks/s3/data_cache/ 0 745 746 -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; diff --git a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql index 24ea62aabf8..a889bea0fcf 100644 --- a/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql +++ b/tests/queries/0_stateless/02240_system_remote_filesystem_cache.sql @@ -2,17 +2,17 @@ -- { echo } -SYSTEM DROP REMOTE FILESYSTEM CACHE; -SET remote_fs_cache_on_write_operations=0; +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'; +CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; SELECT * FROM test FORMAT Null; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; SELECT cache_base_path, file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache; diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference index 1f470d5644f..e83b5551821 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.reference @@ -1,62 +1,62 @@ -- { echo } -SET remote_fs_cache_on_write_operations=1; +SET enable_filesystem_cache_on_write_operations=1; DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -SYSTEM DROP REMOTE FILESYSTEM CACHE; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -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.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +SYSTEM DROP FILESYSTEM CACHE; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path 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) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +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; 0 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 0 -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=1; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path 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) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── file_segment_range_begin: 0 file_segment_range_end: 745 size: 746 state: DOWNLOADED -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.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +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; 7 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 7 -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; 0 SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; 2 SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; 2 -SELECT count() size FROM system.remote_filesystem_cache; +SELECT count() size FROM system.filesystem_cache; 7 -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path 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) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; Row 1: ────── file_segment_range_begin: 0 file_segment_range_end: 1659 size: 1660 state: DOWNLOADED -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.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; +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; 7 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 7 -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 7 -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=0; -SELECT count() FROM system.remote_filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=0; +SELECT count() FROM system.filesystem_cache; 7 INSERT INTO test SELECT number, toString(number) FROM numbers(100); INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000); -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 21 OPTIMIZE TABLE test FINAL; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 27 SET mutations_sync=2; ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; 28 diff --git a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql index cd255d7df51..745af904c5f 100644 --- a/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql +++ b/tests/queries/0_stateless/02241_remote_filesystem_cache_on_insert.sql @@ -2,51 +2,51 @@ -- { echo } -SET remote_fs_cache_on_write_operations=1; +SET enable_filesystem_cache_on_write_operations=1; DROP TABLE IF EXISTS test; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache'; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -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.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.remote_filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path 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) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +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; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=1; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=1; -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -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.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.remote_filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path 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) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +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 count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; SELECT * FROM test FORMAT Null; -SELECT count() FROM system.remote_filesystem_cache WHERE cache_hits > 0; +SELECT count() FROM system.filesystem_cache WHERE cache_hits > 0; -SELECT count() size FROM system.remote_filesystem_cache; +SELECT count() size FROM system.filesystem_cache; -SYSTEM DROP REMOTE FILESYSTEM CACHE; +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200); -SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; -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.remote_filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.remote_filesystem_cache; +SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path 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) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical; +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 count() FROM system.remote_filesystem_cache; -INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS remote_fs_cache_on_write_operations=0; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; +INSERT INTO test SELECT number, toString(number) FROM numbers(100) SETTINGS enable_filesystem_cache_on_write_operations=0; +SELECT count() FROM system.filesystem_cache; INSERT INTO test SELECT number, toString(number) FROM numbers(100); INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000); -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; OPTIMIZE TABLE test FINAL; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache; SET mutations_sync=2; ALTER TABLE test UPDATE value = 'kek' WHERE key = 100; -SELECT count() FROM system.remote_filesystem_cache; +SELECT count() FROM system.filesystem_cache;