From 9f8964a547b5219a580f8b64655b56e2bfa70f61 Mon Sep 17 00:00:00 2001 From: Sharath K S Date: Thu, 26 Sep 2024 04:47:11 -0700 Subject: [PATCH 01/53] Enhance OpenTelemetry span logging to include query settings --- programs/server/config.xml | 4 ++ programs/server/users.xml | 2 + src/Interpreters/executeQuery.cpp | 40 +++++++++++++++++++ .../02421_simple_queries_for_opentelemetry.sh | 14 +++++++ 4 files changed, 60 insertions(+) diff --git a/programs/server/config.xml b/programs/server/config.xml index 10ad831465a..909b8f887a0 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -202,6 +202,10 @@ --> 9009 + + 1 + + + 1 + 1 diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a96350c7ca3..87f15d0a859 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -563,6 +563,46 @@ void logQueryFinish( query_span->addAttributeIfNotZero("clickhouse.written_rows", elem.written_rows); query_span->addAttributeIfNotZero("clickhouse.written_bytes", elem.written_bytes); query_span->addAttributeIfNotZero("clickhouse.memory_usage", elem.memory_usage); + + if (context) + { + std::string user_name = context->getUserName(); + query_span->addAttribute("clickhouse.user", user_name); + } + + if (settings.log_query_settings) + { + for (const auto & setting : settings.allChanged()) + { + auto name = setting.getName(); + Field value = settings.get(name); + String value_str; + + switch (value.getType()) + { + case Field::Types::Which::String: + value_str = value.safeGet(); + break; + case Field::Types::Which::Int64: + value_str = std::to_string(value.safeGet()); + break; + case Field::Types::Which::Float64: + value_str = std::to_string(value.safeGet()); + break; + case Field::Types::Which::Bool: + value_str = value.safeGet() ? "true" : "false"; + break; + case Field::Types::Which::UInt64: + value_str = std::to_string(value.safeGet()); + break; + default: + value_str = "Unsupported type: " + std::to_string(static_cast(value.getType())); + break; + } + + query_span->addAttribute(name, value_str); + } + } query_span->finish(); } } diff --git a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh index 91e85eabcb8..bade298a8cf 100755 --- a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh @@ -44,6 +44,19 @@ ${CLICKHOUSE_CLIENT} -q " ;" } +function check_query_settings() +{ + ${CLICKHOUSE_CLIENT} -q " + SYSTEM FLUSH LOGS; + SELECT attribute + FROM system.opentelemetry_span_log + WHERE finish_date >= yesterday() + AND operation_name = 'query' + AND attribute['clickhouse.query_id'] = '${1}' + FORMAT JSONEachRow; + " +} + # # Set up # @@ -72,6 +85,7 @@ check_query_span "$query_id" query_id=$(${CLICKHOUSE_CLIENT} -q "select generateUUIDv4()"); execute_query $query_id 'select * from opentelemetry_test format Null' check_query_span $query_id +check_query_settings # From 6b3ab311c3be78f29eeebdf334b5d3e4695d97b3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 30 Sep 2024 19:20:37 +0200 Subject: [PATCH 02/53] Allow to cache data for object storage table engines and data lakes using ETag and Path hash for cache key --- programs/server/Server.cpp | 2 + src/Core/Settings.cpp | 1 + src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- .../Cached/CachedObjectStorage.cpp | 2 +- src/IO/ReadSettings.h | 3 + src/Interpreters/Cache/FileCache.cpp | 7 +-- src/Interpreters/Cache/FileCache.h | 2 - src/Interpreters/Cache/FileCacheFactory.cpp | 27 +++++++++ src/Interpreters/Cache/FileCacheFactory.h | 6 ++ src/Interpreters/Cache/FileCacheKey.cpp | 15 +++-- src/Interpreters/Cache/FileCacheKey.h | 10 ++-- src/Interpreters/Cache/FileCache_fwd.h | 2 + src/Interpreters/Context.cpp | 2 + src/Interpreters/tests/gtest_filecache.cpp | 22 ++++---- .../StorageObjectStorageSource.cpp | 55 ++++++++++++++++++- .../System/StorageSystemRemoteDataPaths.cpp | 2 +- .../configs/filesystem_caches.xml | 8 +++ tests/integration/test_storage_s3/test.py | 40 ++++++++++++++ 18 files changed, 175 insertions(+), 33 deletions(-) create mode 100644 tests/integration/test_storage_s3/configs/filesystem_caches.xml diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9cf0e08e0ef..ef47ad17d13 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1486,6 +1486,8 @@ try NamedCollectionFactory::instance().loadIfNot(); + FileCacheFactory::instance().loadDefaultCaches(config()); + /// Initialize main config reloader. std::string include_from_path = config().getString("include_from", "/etc/metrika.xml"); diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 07b4ecd7a24..43c0bf6d77d 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -807,6 +807,7 @@ namespace ErrorCodes M(Bool, enable_filesystem_cache, true, "Use cache for remote filesystem. This setting does not turn on/off cache for disks (must be done via disk config), but allows to bypass cache for some queries if intended", 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, enable_filesystem_cache_log, false, "Allows to record the filesystem caching log for each query", 0) \ + M(String, filesystem_cache_name, "", "Filesystem cache name to use for stateless table engines or data lakes", 0) \ M(Bool, read_from_filesystem_cache_if_exists_otherwise_bypass_cache, false, "Allow to use the filesystem cache in passive mode - benefit from the existing cache entries, but don't put more entries into the cache. If you set this setting for heavy ad-hoc queries and leave it disabled for short real-time queries, this will allows to avoid cache threshing by too heavy queries and to improve the overall system efficiency.", 0) \ M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \ M(UInt64, filesystem_cache_max_download_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be downloaded by a single query", 0) \ diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index c96f5f0c931..3edcb5444f3 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -82,7 +82,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c { if (settings.remote_fs_cache->isInitialized()) { - auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); + auto cache_key = settings.filecache_key.has_value() ? FileCacheKey::fromPath(object_path) : *settings.filecache_key; buf = std::make_unique( object_path, cache_key, diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index ab0d357119c..cf9367661bd 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -31,7 +31,7 @@ CachedObjectStorage::CachedObjectStorage( FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const { - return cache->createKeyForPath(path); + return FileCacheKey::fromPath(path); } ObjectStorageKey diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 7c22682dc76..4f7c962ed9b 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -101,6 +102,7 @@ struct ReadSettings bool enable_filesystem_cache_log = false; size_t filesystem_cache_segments_batch_size = 20; size_t filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = 1000; + std::string filesystem_cache_name; bool use_page_cache_for_disks_without_file_cache = false; bool read_from_page_cache_if_exists_otherwise_bypass_cache = false; @@ -113,6 +115,7 @@ struct ReadSettings size_t remote_read_min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE; FileCachePtr remote_fs_cache; + std::optional filecache_key; /// Bandwidth throttler to use during reading ThrottlerPtr remote_throttler; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ffe9a611014..e0e92f603e7 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -122,11 +122,6 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s query_limit = std::make_unique(); } -FileCache::Key FileCache::createKeyForPath(const String & path) -{ - return Key(path); -} - const FileCache::UserInfo & FileCache::getCommonUser() { static UserInfo user(getCommonUserID(), 0); @@ -1168,7 +1163,7 @@ void FileCache::removeFileSegment(const Key & key, size_t offset, const UserID & void FileCache::removePathIfExists(const String & path, const UserID & user_id) { - removeKeyIfExists(createKeyForPath(path), user_id); + removeKeyIfExists(Key::fromPath(path), user_id); } void FileCache::removeAllReleasable(const UserID & user_id) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 8e8f01ff39e..a25c945cdf7 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -88,8 +88,6 @@ public: const String & getBasePath() const; - static Key createKeyForPath(const String & path); - static const UserInfo & getCommonUser(); static const UserInfo & getInternalUser(); diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index 6f0ba7bffaf..c638f445c36 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -1,5 +1,6 @@ #include "FileCacheFactory.h" #include "FileCache.h" +#include namespace DB { @@ -43,6 +44,16 @@ FileCacheFactory::CacheByName FileCacheFactory::getAll() return caches_by_name; } +FileCachePtr FileCacheFactory::get(const std::string & cache_name) +{ + std::lock_guard lock(mutex); + + auto it = caches_by_name.find(cache_name); + if (it == caches_by_name.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no cache by name `{}`", cache_name); + return it->second->cache; +} + FileCachePtr FileCacheFactory::getOrCreate( const std::string & cache_name, const FileCacheSettings & file_cache_settings, @@ -202,4 +213,20 @@ void FileCacheFactory::clear() caches_by_name.clear(); } +void FileCacheFactory::loadDefaultCaches(const Poco::Util::AbstractConfiguration & config) +{ + Poco::Util::AbstractConfiguration::Keys cache_names; + config.keys(FILECACHE_DEFAULT_CONFIG_PATH, cache_names); + auto * log = &Poco::Logger::get("FileCacheFactory"); + LOG_DEBUG(log, "Will load {} caches from default cache config", cache_names.size()); + for (const auto & name : cache_names) + { + FileCacheSettings settings; + const auto & config_path = fmt::format("{}.{}", FILECACHE_DEFAULT_CONFIG_PATH, name); + settings.loadFromConfig(config, config_path); + auto cache = getOrCreate(name, settings, config_path); + cache->initialize(); + LOG_DEBUG(log, "Loaded cache `{}` from default cache config", name); + } +} } diff --git a/src/Interpreters/Cache/FileCacheFactory.h b/src/Interpreters/Cache/FileCacheFactory.h index 350932dce36..d770823782e 100644 --- a/src/Interpreters/Cache/FileCacheFactory.h +++ b/src/Interpreters/Cache/FileCacheFactory.h @@ -44,6 +44,8 @@ public: const FileCacheSettings & file_cache_settings, const std::string & config_path); + FileCachePtr get(const std::string & cache_name); + FileCachePtr create( const std::string & cache_name, const FileCacheSettings & file_cache_settings, @@ -53,8 +55,12 @@ public: FileCacheDataPtr getByName(const std::string & cache_name); + void loadDefaultCaches(const Poco::Util::AbstractConfiguration & config); + void updateSettingsFromConfig(const Poco::Util::AbstractConfiguration & config); + void remove(FileCachePtr cache); + void clear(); private: diff --git a/src/Interpreters/Cache/FileCacheKey.cpp b/src/Interpreters/Cache/FileCacheKey.cpp index 75a8ac2934e..ffa1bb4b96a 100644 --- a/src/Interpreters/Cache/FileCacheKey.cpp +++ b/src/Interpreters/Cache/FileCacheKey.cpp @@ -12,11 +12,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -FileCacheKey::FileCacheKey(const std::string & path) - : key(sipHash128(path.data(), path.size())) -{ -} - FileCacheKey::FileCacheKey(const UInt128 & key_) : key(key_) { @@ -32,6 +27,16 @@ FileCacheKey FileCacheKey::random() return FileCacheKey(UUIDHelpers::generateV4().toUnderType()); } +FileCacheKey FileCacheKey::fromPath(const std::string & path) +{ + return FileCacheKey(sipHash128(path.data(), path.size())); +} + +FileCacheKey FileCacheKey::fromKey(const UInt128 & key) +{ + return FileCacheKey(key); +} + FileCacheKey FileCacheKey::fromKeyString(const std::string & key_str) { if (key_str.size() != 32) diff --git a/src/Interpreters/Cache/FileCacheKey.h b/src/Interpreters/Cache/FileCacheKey.h index 7de2a8f7800..71b8a8745ee 100644 --- a/src/Interpreters/Cache/FileCacheKey.h +++ b/src/Interpreters/Cache/FileCacheKey.h @@ -14,16 +14,16 @@ struct FileCacheKey FileCacheKey() = default; - explicit FileCacheKey(const std::string & path); - - explicit FileCacheKey(const UInt128 & key_); - static FileCacheKey random(); + static FileCacheKey fromPath(const std::string & path); + static FileCacheKey fromKey(const UInt128 & key); + static FileCacheKey fromKeyString(const std::string & key_str); bool operator==(const FileCacheKey & other) const { return key == other.key; } bool operator<(const FileCacheKey & other) const { return key < other.key; } - static FileCacheKey fromKeyString(const std::string & key_str); +private: + explicit FileCacheKey(const UInt128 & key_); }; using FileCacheKeyAndOffset = std::pair; diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 3bb334dbf05..bdd591d75da 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -15,10 +15,12 @@ static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; static constexpr double FILECACHE_DEFAULT_FREE_SPACE_SIZE_RATIO = 0; /// Disabled. static constexpr double FILECACHE_DEFAULT_FREE_SPACE_ELEMENTS_RATIO = 0; /// Disabled. static constexpr int FILECACHE_DEFAULT_FREE_SPACE_REMOVE_BATCH = 10; +static constexpr auto FILECACHE_DEFAULT_CONFIG_PATH = "filesystem_caches"; class FileCache; using FileCachePtr = std::shared_ptr; struct FileCacheSettings; +struct FileCacheKey; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 25e31c31cb0..d1bc7b977c8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -186,6 +186,7 @@ namespace Setting extern const SettingsUInt64 backup_threads; extern const SettingsString cluster_for_parallel_replicas; extern const SettingsBool enable_filesystem_cache; + extern const SettingsString filesystem_cache_name; extern const SettingsBool enable_filesystem_cache_log; extern const SettingsBool enable_filesystem_cache_on_write_operations; extern const SettingsBool enable_filesystem_read_prefetches_log; @@ -5638,6 +5639,7 @@ ReadSettings Context::getReadSettings() const res.filesystem_cache_segments_batch_size = settings_ref[Setting::filesystem_cache_segments_batch_size]; res.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = settings_ref[Setting::filesystem_cache_reserve_space_wait_lock_timeout_milliseconds]; + res.filesystem_cache_name = settings_ref[Setting::filesystem_cache_name]; res.filesystem_cache_max_download_size = settings_ref[Setting::filesystem_cache_max_download_size]; res.skip_download_if_exceeds_query_cache = settings_ref[Setting::skip_download_if_exceeds_query_cache]; diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index ce5f3b45781..007b31d9fdc 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -372,7 +372,7 @@ TEST_F(FileCacheTest, LRUPolicy) std::cerr << "Step 1\n"; auto cache = DB::FileCache("1", settings); cache.initialize(); - auto key = DB::FileCache::createKeyForPath("key1"); + auto key = DB::FileCacheKey::fromPath("key1"); auto get_or_set = [&](size_t offset, size_t size) { @@ -736,7 +736,7 @@ TEST_F(FileCacheTest, LRUPolicy) auto cache2 = DB::FileCache("2", settings); cache2.initialize(); - auto key = DB::FileCache::createKeyForPath("key1"); + auto key = DB::FileCacheKey::fromPath("key1"); /// Get [2, 29] assertEqual( @@ -755,7 +755,7 @@ TEST_F(FileCacheTest, LRUPolicy) fs::create_directories(settings2.base_path); auto cache2 = DB::FileCache("3", settings2); cache2.initialize(); - auto key = DB::FileCache::createKeyForPath("key1"); + auto key = DB::FileCacheKey::fromPath("key1"); /// Get [0, 24] assertEqual( @@ -770,7 +770,7 @@ TEST_F(FileCacheTest, LRUPolicy) auto cache = FileCache("4", settings); cache.initialize(); - const auto key = FileCache::createKeyForPath("key10"); + const auto key = FileCacheKey::fromPath("key10"); const auto key_path = cache.getKeyPath(key, user); cache.removeAllReleasable(user.user_id); @@ -794,7 +794,7 @@ TEST_F(FileCacheTest, LRUPolicy) auto cache = DB::FileCache("5", settings); cache.initialize(); - const auto key = FileCache::createKeyForPath("key10"); + const auto key = FileCacheKey::fromPath("key10"); const auto key_path = cache.getKeyPath(key, user); cache.removeAllReleasable(user.user_id); @@ -833,7 +833,7 @@ TEST_F(FileCacheTest, writeBuffer) segment_settings.kind = FileSegmentKind::Ephemeral; segment_settings.unbounded = true; - auto cache_key = FileCache::createKeyForPath(key); + auto cache_key = FileCacheKey::fromPath(key); auto holder = cache.set(cache_key, 0, 3, segment_settings, user); /// The same is done in TemporaryDataOnDisk::createStreamToCacheFile. std::filesystem::create_directories(cache.getKeyPath(cache_key, user)); @@ -961,7 +961,7 @@ TEST_F(FileCacheTest, temporaryData) const auto user = FileCache::getCommonUser(); auto tmp_data_scope = std::make_shared(nullptr, &file_cache, TemporaryDataOnDiskSettings{}); - auto some_data_holder = file_cache.getOrSet(FileCache::createKeyForPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user); + auto some_data_holder = file_cache.getOrSet(FileCacheKey::fromPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user); { ASSERT_EQ(some_data_holder->size(), 5); @@ -1103,7 +1103,7 @@ TEST_F(FileCacheTest, CachedReadBuffer) auto cache = std::make_shared("8", settings); cache->initialize(); - auto key = cache->createKeyForPath(file_path); + auto key = DB::FileCacheKey::fromPath(file_path); const auto user = FileCache::getCommonUser(); { @@ -1219,7 +1219,7 @@ TEST_F(FileCacheTest, SLRUPolicy) { auto cache = DB::FileCache(std::to_string(++file_cache_name), settings); cache.initialize(); - auto key = FileCache::createKeyForPath("key1"); + auto key = FileCacheKey::fromPath("key1"); auto add_range = [&](size_t offset, size_t size) { @@ -1342,7 +1342,7 @@ TEST_F(FileCacheTest, SLRUPolicy) std::string data1(15, '*'); auto file1 = write_file("test1", data1); - auto key1 = cache->createKeyForPath(file1); + auto key1 = DB::FileCacheKey::fromPath(file1); read_and_check(file1, key1, data1); @@ -1358,7 +1358,7 @@ TEST_F(FileCacheTest, SLRUPolicy) std::string data2(10, '*'); auto file2 = write_file("test2", data2); - auto key2 = cache->createKeyForPath(file2); + auto key2 = DB::FileCacheKey::fromPath(file2); read_and_check(file2, key2, data2); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 641b43e57d6..ded4babdb46 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -7,6 +7,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -418,13 +421,38 @@ std::future StorageObjectStorageSource return create_reader_scheduler([=, this] { return createReader(); }, Priority{}); } +static void patchReadSettings( + ReadSettings & read_settings, + const StorageObjectStorage::ObjectInfo & object_info, + const LoggerPtr & log) +{ + if (read_settings.enable_filesystem_cache && !read_settings.filesystem_cache_name.empty()) + { + if (object_info.metadata->etag.empty()) + { + LOG_WARNING(log, "Cannot use filesystem cache, no etag specified"); + } + else + { + SipHash hash; + hash.update(object_info.getPath()); + hash.update(object_info.metadata->etag); + read_settings.filecache_key = FileCacheKey::fromKey(hash.get128()); + read_settings.remote_fs_cache = FileCacheFactory::instance().get(read_settings.filesystem_cache_name); + + LOG_TEST(log, "Using filesystem cache `{}` (path: {}, etag: {}, hash: {})", + read_settings.filesystem_cache_name, object_info.getPath(), + object_info.metadata->etag, toString(hash.get128())); + } + } +} + std::unique_ptr StorageObjectStorageSource::createReadBuffer( const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) { const auto & object_size = object_info.metadata->size_bytes; auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; /// FIXME: Changing this setting to default value breaks something around parquet reading read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; @@ -434,6 +462,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( /// User's object may change, don't cache it. read_settings.use_page_cache_for_disks_without_file_cache = false; + patchReadSettings(read_settings, object_info, log); + // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. @@ -452,6 +482,29 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( } else { + if (!read_settings.filesystem_cache_name.empty()) + { + /// TODO: fix inconsistency with readObject and readObjects... + + auto read_buffer_creator = [&]() + { + return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + }; + return std::make_unique( + object_info.getPath(), + read_settings.filecache_key.value(), + read_settings.remote_fs_cache, + FileCache::getCommonUser(), + read_buffer_creator, + read_settings, + std::string(CurrentThread::getQueryId()), + object_info.metadata->size_bytes, + /* allow_seeks */false, + /* use_external_buffer */true, + /* read_until_position */std::nullopt, + nullptr); + } + /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); } diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 783efee4cc4..b10fa4b1b75 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -401,7 +401,7 @@ Chunk SystemRemoteDataPathsSource::generate() if (cache) { - auto cache_paths = cache->tryGetCachePaths(cache->createKeyForPath(object.remote_path)); + auto cache_paths = cache->tryGetCachePaths(FileCacheKey::fromPath(object.remote_path)); col_cache_paths->insert(Array(cache_paths.begin(), cache_paths.end())); } else diff --git a/tests/integration/test_storage_s3/configs/filesystem_caches.xml b/tests/integration/test_storage_s3/configs/filesystem_caches.xml new file mode 100644 index 00000000000..e91362640fe --- /dev/null +++ b/tests/integration/test_storage_s3/configs/filesystem_caches.xml @@ -0,0 +1,8 @@ + + + + 1Gi + cache1 + + + diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ab327afe90b..695accc4176 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -55,6 +55,7 @@ def started_cluster(): "configs/named_collections.xml", "configs/schema_cache.xml", "configs/blob_log.xml", + "configs/filesystem_caches.xml", ], user_configs=[ "configs/access.xml", @@ -2393,3 +2394,42 @@ def test_respect_object_existence_on_partitioned_write(started_cluster): ) assert int(result) == 44 + + +def test_filesystem_cache(started_cluster): + id = uuid.uuid4() + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + table_name = "test_filesystem_cache" + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{table_name}.tsv', auto, 'x UInt64') select number from numbers(100) SETTINGS s3_truncate_on_insert=1" + ) + + query_id = f"{table_name}-{uuid.uuid4()}" + instance.query( + f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{table_name}.tsv') SETTINGS filesystem_cache_name = 'cache1', enable_filesystem_cache=1", + query_id=query_id, + ) + + instance.query("SYSTEM FLUSH LOGS") + + count = int( + instance.query( + f"SELECT ProfileEvents['CachedReadBufferCacheWriteBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) + + assert count == 290 + + query_id = f"{table_name}-{uuid.uuid4()}" + instance.query( + f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{table_name}.tsv') SETTINGS filesystem_cache_name = 'cache1', enable_filesystem_cache=1", + query_id=query_id, + ) + + instance.query("SYSTEM FLUSH LOGS") + + assert count == int(instance.query( + f"SELECT ProfileEvents['CachedReadBufferReadFromCacheBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + )) From 69cd9871a4402539a2816480da7fa96519895333 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 30 Sep 2024 21:16:11 +0200 Subject: [PATCH 03/53] Update settings changes history --- src/Core/SettingsChangesHistory.cpp | 1 + tests/integration/test_storage_s3/test.py | 8 +++++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c9723deaad8..7c3747382f5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -70,6 +70,7 @@ static std::initializer_list Date: Tue, 1 Oct 2024 03:27:00 -0700 Subject: [PATCH 04/53] Fixed review comments Removed config files changes --- programs/server/config.xml | 4 -- programs/server/users.xml | 2 - ...simple_queries_for_opentelemetry.reference | 2 + .../02421_simple_queries_for_opentelemetry.sh | 46 ++++++++++++++----- 4 files changed, 36 insertions(+), 18 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 909b8f887a0..10ad831465a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -202,10 +202,6 @@ --> 9009 - - 1 - - - 1 - 1 diff --git a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.reference b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.reference index d167d905636..9567f89b091 100644 --- a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.reference +++ b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.reference @@ -2,3 +2,5 @@ {"query":"show databases format Null\n "} {"query":"insert into opentelemetry_test values","read_rows":"3","written_rows":"3"} {"query":"select * from opentelemetry_test format Null\n ","read_rows":"3","written_rows":""} +{"query":"SELECT * FROM opentelemetry_test FORMAT Null\n ","read_rows":"3","written_rows":""} +{"min_compress_block_size":"present","max_block_size":"present","max_execution_time":"present"} diff --git a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh index bade298a8cf..9f94183087e 100755 --- a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh @@ -44,17 +44,35 @@ ${CLICKHOUSE_CLIENT} -q " ;" } -function check_query_settings() -{ - ${CLICKHOUSE_CLIENT} -q " - SYSTEM FLUSH LOGS; - SELECT attribute - FROM system.opentelemetry_span_log - WHERE finish_date >= yesterday() - AND operation_name = 'query' - AND attribute['clickhouse.query_id'] = '${1}' - FORMAT JSONEachRow; - " +function check_query_settings() { + result=$(${CLICKHOUSE_CLIENT} -q " + SYSTEM FLUSH LOGS; + SELECT + attribute['min_compress_block_size'], + attribute['max_block_size'], + attribute['max_execution_time'] + FROM system.opentelemetry_span_log + WHERE finish_date >= yesterday() + AND operation_name = 'query' + AND attribute['clickhouse.query_id'] = '${1}' + FORMAT JSONEachRow; + ") + + local min_present="not found" + local max_present="not found" + local execution_time_present="not found" + + if [[ $result == *"min_compress_block_size"* ]]; then + min_present="present" + fi + if [[ $result == *"max_block_size"* ]]; then + max_present="present" + fi + if [[ $result == *"max_execution_time"* ]]; then + execution_time_present="present" + fi + + echo "{\"min_compress_block_size\":\"$min_present\",\"max_block_size\":\"$max_present\",\"max_execution_time\":\"$execution_time_present\"}" } # @@ -85,8 +103,12 @@ check_query_span "$query_id" query_id=$(${CLICKHOUSE_CLIENT} -q "select generateUUIDv4()"); execute_query $query_id 'select * from opentelemetry_test format Null' check_query_span $query_id -check_query_settings +# Test 5: A normal select query with a setting +query_id=$(${CLICKHOUSE_CLIENT} -q "SELECT generateUUIDv4() SETTINGS max_execution_time=3600") +execute_query "$query_id" 'SELECT * FROM opentelemetry_test FORMAT Null' +check_query_span "$query_id" +check_query_settings "$query_id" "max_execution_time" # # Tear down From 4eddbe6fc2284fe05d98073a69d32a9158d3f1dd Mon Sep 17 00:00:00 2001 From: Sharath K S Date: Tue, 1 Oct 2024 04:47:38 -0700 Subject: [PATCH 05/53] Fix indentation in shell script --- .../02421_simple_queries_for_opentelemetry.sh | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh index 9f94183087e..e36c8c36cbf 100755 --- a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh @@ -44,35 +44,35 @@ ${CLICKHOUSE_CLIENT} -q " ;" } -function check_query_settings() { - result=$(${CLICKHOUSE_CLIENT} -q " - SYSTEM FLUSH LOGS; - SELECT - attribute['min_compress_block_size'], - attribute['max_block_size'], - attribute['max_execution_time'] - FROM system.opentelemetry_span_log - WHERE finish_date >= yesterday() - AND operation_name = 'query' - AND attribute['clickhouse.query_id'] = '${1}' - FORMAT JSONEachRow; +function check_query_settings() +{ +result=$(${CLICKHOUSE_CLIENT} -q " + SYSTEM FLUSH LOGS; + SELECT attribute['min_compress_block_size'], + attribute['max_block_size'], + attribute['max_execution_time'] + FROM system.opentelemetry_span_log + WHERE finish_date >= yesterday() + AND operation_name = 'query' + AND attribute['clickhouse.query_id'] = '${1}' + FORMAT JSONEachRow; ") - local min_present="not found" - local max_present="not found" - local execution_time_present="not found" + local min_present="not found" + local max_present="not found" + local execution_time_present="not found" - if [[ $result == *"min_compress_block_size"* ]]; then - min_present="present" - fi - if [[ $result == *"max_block_size"* ]]; then - max_present="present" - fi - if [[ $result == *"max_execution_time"* ]]; then - execution_time_present="present" - fi + if [[ $result == *"min_compress_block_size"* ]]; then + min_present="present" + fi + if [[ $result == *"max_block_size"* ]]; then + max_present="present" + fi + if [[ $result == *"max_execution_time"* ]]; then + execution_time_present="present" + fi - echo "{\"min_compress_block_size\":\"$min_present\",\"max_block_size\":\"$max_present\",\"max_execution_time\":\"$execution_time_present\"}" + echo "{\"min_compress_block_size\":\"$min_present\",\"max_block_size\":\"$max_present\",\"max_execution_time\":\"$execution_time_present\"}" } # From d6acfd5dd32a281957ad77015c04b6941f0ad567 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Oct 2024 13:19:40 +0200 Subject: [PATCH 06/53] Adjust code to merged refactoring from master --- .../Cached/CachedObjectStorage.cpp | 2 +- .../StorageObjectStorageSource.cpp | 69 +++++++++++-------- 2 files changed, 41 insertions(+), 30 deletions(-) diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 043f84345cb..163ff3a9c68 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -71,7 +71,7 @@ std::unique_ptr CachedObjectStorage::readObject( /// NOL { if (cache->isInitialized()) { - auto cache_key = cache->createKeyForPath(object.remote_path); + auto cache_key = FileCacheKey::fromPath(object.remote_path); auto global_context = Context::getGlobalContextInstance(); auto modified_read_settings = read_settings.withNestedBuffer(); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 740d214a0f6..d505af7c646 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -424,32 +424,6 @@ std::future StorageObjectStorageSource return create_reader_scheduler([=, this] { return createReader(); }, Priority{}); } -static void patchReadSettings( - ReadSettings & read_settings, - const StorageObjectStorage::ObjectInfo & object_info, - const LoggerPtr & log) -{ - if (read_settings.enable_filesystem_cache && !read_settings.filesystem_cache_name.empty()) - { - if (object_info.metadata->etag.empty()) - { - LOG_WARNING(log, "Cannot use filesystem cache, no etag specified"); - } - else - { - SipHash hash; - hash.update(object_info.getPath()); - hash.update(object_info.metadata->etag); - read_settings.filecache_key = FileCacheKey::fromKey(hash.get128()); - read_settings.remote_fs_cache = FileCacheFactory::instance().get(read_settings.filesystem_cache_name); - - LOG_TEST(log, "Using filesystem cache `{}` (path: {}, etag: {}, hash: {})", - read_settings.filesystem_cache_name, object_info.getPath(), - object_info.metadata->etag, toString(hash.get128())); - } - } -} - std::unique_ptr StorageObjectStorageSource::createReadBuffer( const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) { @@ -459,7 +433,6 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( /// FIXME: Changing this setting to default value breaks something around parquet reading read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; /// User's object may change, don't cache it. - read_settings.enable_filesystem_cache = false; read_settings.use_page_cache_for_disks_without_file_cache = false; const bool object_too_small = object_size <= 2 * context_->getSettingsRef()[Setting::max_download_buffer_size]; @@ -472,14 +445,52 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( auto impl = object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); - patchReadSettings(read_settings, object_info, log); - // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. if (!use_prefetch) return impl; + if (read_settings.enable_filesystem_cache && !read_settings.filesystem_cache_name.empty()) + { + if (object_info.metadata->etag.empty()) + { + LOG_WARNING(log, "Cannot use filesystem cache, no etag specified"); + } + else + { + SipHash hash; + hash.update(object_info.getPath()); + hash.update(object_info.metadata->etag); + + const auto cache_key = FileCacheKey::fromKey(hash.get128()); + auto cache = FileCacheFactory::instance().get(read_settings.filesystem_cache_name); + + auto read_buffer_creator = [path = object_info.getPath(), object_size, read_settings, object_storage]() + { + return object_storage->readObject(StoredObject(path, "", object_size), read_settings); + }; + + impl = std::make_unique( + object_info.getPath(), + cache_key, + cache, + FileCache::getCommonUser(), + read_buffer_creator, + read_settings, + std::string(CurrentThread::getQueryId()), + object_size, + /* allow_seeks */false, + /* use_external_buffer */true, + /* read_until_position */std::nullopt, + context_->getFilesystemCacheLog()); + + LOG_TEST(log, "Using filesystem cache `{}` (path: {}, etag: {}, hash: {})", + read_settings.filesystem_cache_name, object_info.getPath(), + object_info.metadata->etag, toString(hash.get128())); + } + } + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); From 24c215b6c7bec05009108198e4630870faa27d67 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Oct 2024 17:02:49 +0200 Subject: [PATCH 07/53] Use cached buffer in ramaining places --- src/IO/ReadSettings.h | 2 -- src/Interpreters/Context.cpp | 2 -- .../DataLakes/DeltaLakeMetadata.cpp | 10 +++++++--- .../DataLakes/IcebergMetadata.cpp | 18 ++++++++++++------ .../ObjectStorage/ReadBufferIterator.cpp | 8 ++------ .../StorageObjectStorageSource.cpp | 14 ++++++++------ .../ObjectStorage/StorageObjectStorageSource.h | 10 +++++----- 7 files changed, 34 insertions(+), 30 deletions(-) diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 41d484e2922..8a42360fd30 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -106,7 +105,6 @@ struct ReadSettings bool enable_filesystem_cache_log = false; size_t filesystem_cache_segments_batch_size = 20; size_t filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = 1000; - std::string filesystem_cache_name; bool use_page_cache_for_disks_without_file_cache = false; bool read_from_page_cache_if_exists_otherwise_bypass_cache = false; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5f6b72ce553..3caebeb0ea5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -186,7 +186,6 @@ namespace Setting extern const SettingsUInt64 backup_threads; extern const SettingsString cluster_for_parallel_replicas; extern const SettingsBool enable_filesystem_cache; - extern const SettingsString filesystem_cache_name; extern const SettingsBool enable_filesystem_cache_log; extern const SettingsBool enable_filesystem_cache_on_write_operations; extern const SettingsBool enable_filesystem_read_prefetches_log; @@ -5642,7 +5641,6 @@ ReadSettings Context::getReadSettings() const res.filesystem_cache_segments_batch_size = settings_ref[Setting::filesystem_cache_segments_batch_size]; res.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = settings_ref[Setting::filesystem_cache_reserve_space_wait_lock_timeout_milliseconds]; - res.filesystem_cache_name = settings_ref[Setting::filesystem_cache_name]; res.filesystem_cache_max_download_size = settings_ref[Setting::filesystem_cache_max_download_size]; res.skip_download_if_exceeds_query_cache = settings_ref[Setting::skip_download_if_exceeds_query_cache]; diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index f04e868ee5a..6242b2336b9 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -185,7 +186,8 @@ struct DeltaLakeMetadataImpl std::set & result) { auto read_settings = context->getReadSettings(); - auto buf = object_storage->readObject(StoredObject(metadata_file_path), read_settings); + StorageObjectStorageSource::ObjectInfo object_info(metadata_file_path); + auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, context, log); char c; while (!buf->eof()) @@ -492,7 +494,8 @@ struct DeltaLakeMetadataImpl String json_str; auto read_settings = context->getReadSettings(); - auto buf = object_storage->readObject(StoredObject(last_checkpoint_file), read_settings); + StorageObjectStorageSource::ObjectInfo object_info(last_checkpoint_file); + auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, context, log); readJSONObjectPossiblyInvalid(json_str, *buf); const JSON json(json_str); @@ -557,7 +560,8 @@ struct DeltaLakeMetadataImpl LOG_TRACE(log, "Using checkpoint file: {}", checkpoint_path.string()); auto read_settings = context->getReadSettings(); - auto buf = object_storage->readObject(StoredObject(checkpoint_path), read_settings); + StorageObjectStorageSource::ObjectInfo object_info(checkpoint_path); + auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, context, log); auto format_settings = getFormatSettings(context); /// Force nullable, because this parquet file for some reason does not have nullable diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index ffc4dd09a3a..2281daef50f 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -387,9 +388,13 @@ DataLakeMetadataPtr IcebergMetadata::create( ContextPtr local_context) { const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration); - LOG_DEBUG(getLogger("IcebergMetadata"), "Parse metadata {}", metadata_file_path); - auto read_settings = local_context->getReadSettings(); - auto buf = object_storage->readObject(StoredObject(metadata_file_path), read_settings); + + auto log = getLogger("IcebergMetadata"); + LOG_DEBUG(log, "Parse metadata {}", metadata_file_path); + + StorageObjectStorageSource::ObjectInfo object_info(metadata_file_path); + auto buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, local_context, log); + String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); @@ -456,8 +461,8 @@ Strings IcebergMetadata::getDataFiles() const LOG_TEST(log, "Collect manifest files from manifest list {}", manifest_list_file); auto context = getContext(); - auto read_settings = context->getReadSettings(); - auto manifest_list_buf = object_storage->readObject(StoredObject(manifest_list_file), read_settings); + StorageObjectStorageSource::ObjectInfo object_info(manifest_list_file); + auto manifest_list_buf = StorageObjectStorageSource::createReadBuffer(object_info, object_storage, context, log); auto manifest_list_file_reader = std::make_unique(std::make_unique(*manifest_list_buf)); auto data_type = AvroSchemaReader::avroNodeToDataType(manifest_list_file_reader->dataSchema().root()->leafAt(0)); @@ -487,7 +492,8 @@ Strings IcebergMetadata::getDataFiles() const { LOG_TEST(log, "Process manifest file {}", manifest_file); - auto buffer = object_storage->readObject(StoredObject(manifest_file), read_settings); + StorageObjectStorageSource::ObjectInfo manifest_object_info(manifest_file); + auto buffer = StorageObjectStorageSource::createReadBuffer(manifest_object_info, object_storage, context, log); auto manifest_file_reader = std::make_unique(std::make_unique(*buffer)); /// Manifest file should always have table schema in avro file metadata. By now we don't support tables with evolved schema, diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index f0f3de97652..60eab93c8da 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -151,7 +151,7 @@ std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() auto context = getContext(); const auto & path = current_object_info->isArchive() ? current_object_info->getPathToArchive() : current_object_info->getPath(); - auto impl = object_storage->readObject(StoredObject(path), context->getReadSettings()); + auto impl = StorageObjectStorageSource::createReadBuffer(*current_object_info, object_storage, context, getLogger("ReadBufferIterator")); const auto compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); const auto zstd_window = static_cast(context->getSettingsRef()[Setting::zstd_window_log_max]); @@ -277,11 +277,7 @@ ReadBufferIterator::Data ReadBufferIterator::next() else { compression_method = chooseCompressionMethod(filename, configuration->compression_method); - read_buf = object_storage->readObject( - StoredObject(current_object_info->getPath()), - getContext()->getReadSettings(), - {}, - current_object_info->metadata->size_bytes); + read_buf = StorageObjectStorageSource::createReadBuffer(*current_object_info, object_storage, getContext(), getLogger("ReadBufferIterator")); } if (!query_settings.skip_empty_files || !read_buf->eof()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index d505af7c646..ce1f354f90f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -40,6 +40,7 @@ namespace Setting extern const SettingsUInt64 max_download_buffer_size; extern const SettingsMaxThreads max_threads; extern const SettingsBool use_cache_for_count_from_files; + extern const SettingsString filesystem_cache_name; } namespace ErrorCodes @@ -424,10 +425,11 @@ std::future StorageObjectStorageSource return create_reader_scheduler([=, this] { return createReader(); }, Priority{}); } -std::unique_ptr StorageObjectStorageSource::createReadBuffer( +std::unique_ptr StorageObjectStorageSource::createReadBuffer( const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) { const auto & object_size = object_info.metadata->size_bytes; + const auto & settings = context_->getSettingsRef(); auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); /// FIXME: Changing this setting to default value breaks something around parquet reading @@ -451,7 +453,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( if (!use_prefetch) return impl; - if (read_settings.enable_filesystem_cache && !read_settings.filesystem_cache_name.empty()) + const auto filesystem_cache_name = settings[Setting::filesystem_cache_name].value; + if (read_settings.enable_filesystem_cache && !filesystem_cache_name.empty()) { if (object_info.metadata->etag.empty()) { @@ -464,7 +467,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( hash.update(object_info.metadata->etag); const auto cache_key = FileCacheKey::fromKey(hash.get128()); - auto cache = FileCacheFactory::instance().get(read_settings.filesystem_cache_name); + auto cache = FileCacheFactory::instance().get(filesystem_cache_name); auto read_buffer_creator = [path = object_info.getPath(), object_size, read_settings, object_storage]() { @@ -486,7 +489,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( context_->getFilesystemCacheLog()); LOG_TEST(log, "Using filesystem cache `{}` (path: {}, etag: {}, hash: {})", - read_settings.filesystem_cache_name, object_info.getPath(), + filesystem_cache_name, object_info.getPath(), object_info.metadata->etag, toString(hash.get128())); } } @@ -831,8 +834,7 @@ StorageObjectStorageSource::ArchiveIterator::createArchiveReader(ObjectInfoPtr o /* path_to_archive */object_info->getPath(), /* archive_read_function */[=, this]() { - StoredObject stored_object(object_info->getPath(), "", size); - return object_storage->readObject(stored_object, getContext()->getReadSettings()); + return StorageObjectStorageSource::createReadBuffer(*object_info, object_storage, getContext(), logger); }, /* archive_size */size); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 8ee3b023638..cd5bcbb32ff 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -66,6 +66,11 @@ public: const ObjectInfo & object_info, bool include_connection_info = true); + static std::unique_ptr createReadBuffer( + const ObjectInfo & object_info, + const ObjectStoragePtr & object_storage, + const ContextPtr & context_, + const LoggerPtr & log); protected: const String name; ObjectStoragePtr object_storage; @@ -135,11 +140,6 @@ protected: ReaderHolder createReader(); std::future createReaderAsync(); - static std::unique_ptr createReadBuffer( - const ObjectInfo & object_info, - const ObjectStoragePtr & object_storage, - const ContextPtr & context_, - const LoggerPtr & log); void addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows); void lazyInitialize(); From 3d2814f05d2fa1d76b3a16739774f5a442d07dc2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Oct 2024 17:42:50 +0200 Subject: [PATCH 08/53] Add a test for iceberg --- .../StorageObjectStorageSource.cpp | 77 ++++++++++++------- .../StorageObjectStorageSource.h | 2 +- .../configs/config.d/filesystem_caches.xml | 8 ++ .../integration/test_storage_iceberg/test.py | 58 +++++++++++++- 4 files changed, 117 insertions(+), 28 deletions(-) create mode 100644 tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index ce1f354f90f..0032a50fc73 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -426,35 +426,51 @@ std::future StorageObjectStorageSource } std::unique_ptr StorageObjectStorageSource::createReadBuffer( - const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) + ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) { - const auto & object_size = object_info.metadata->size_bytes; const auto & settings = context_->getSettingsRef(); + const auto & read_settings = context_->getReadSettings(); - auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); + const auto filesystem_cache_name = settings[Setting::filesystem_cache_name].value; + bool use_cache = read_settings.enable_filesystem_cache + && !filesystem_cache_name.empty() + && (object_storage->getType() == ObjectStorageType::Azure + || object_storage->getType() == ObjectStorageType::S3); + + if (!object_info.metadata) + { + if (!use_cache) + { + return object_storage->readObject(StoredObject(object_info.getPath()), read_settings); + } + object_info.metadata = object_storage->getObjectMetadata(object_info.getPath()); + } + + const auto & object_size = object_info.metadata->size_bytes; + + auto modified_read_settings = read_settings.adjustBufferSize(object_size); /// FIXME: Changing this setting to default value breaks something around parquet reading - read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; + modified_read_settings.remote_read_min_bytes_for_seek = modified_read_settings.remote_fs_buffer_size; /// User's object may change, don't cache it. - read_settings.use_page_cache_for_disks_without_file_cache = false; - - const bool object_too_small = object_size <= 2 * context_->getSettingsRef()[Setting::max_download_buffer_size]; - const bool use_prefetch = object_too_small - && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool - && read_settings.remote_fs_prefetch; - - if (use_prefetch) - read_settings.remote_read_buffer_use_external_buffer = true; - - auto impl = object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + modified_read_settings.use_page_cache_for_disks_without_file_cache = false; // Create a read buffer that will prefetch the first ~1 MB of the file. // When reading lots of tiny files, this prefetching almost doubles the throughput. // For bigger files, parallel reading is more useful. - if (!use_prefetch) - return impl; + const bool object_too_small = object_size <= 2 * context_->getSettingsRef()[Setting::max_download_buffer_size]; + const bool use_prefetch = object_too_small + && modified_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool + && modified_read_settings.remote_fs_prefetch; - const auto filesystem_cache_name = settings[Setting::filesystem_cache_name].value; - if (read_settings.enable_filesystem_cache && !filesystem_cache_name.empty()) + /// FIXME: Use async buffer if use_cache, + /// because CachedOnDiskReadBufferFromFile does not work as an independent buffer currently. + const bool use_async_buffer = use_prefetch || use_cache; + + if (use_async_buffer) + modified_read_settings.remote_read_buffer_use_external_buffer = true; + + std::unique_ptr impl; + if (use_cache) { if (object_info.metadata->etag.empty()) { @@ -469,9 +485,9 @@ std::unique_ptr StorageObjectStorageSource::createReadBu const auto cache_key = FileCacheKey::fromKey(hash.get128()); auto cache = FileCacheFactory::instance().get(filesystem_cache_name); - auto read_buffer_creator = [path = object_info.getPath(), object_size, read_settings, object_storage]() + auto read_buffer_creator = [path = object_info.getPath(), object_size, modified_read_settings, object_storage]() { - return object_storage->readObject(StoredObject(path, "", object_size), read_settings); + return object_storage->readObject(StoredObject(path, "", object_size), modified_read_settings); }; impl = std::make_unique( @@ -480,10 +496,10 @@ std::unique_ptr StorageObjectStorageSource::createReadBu cache, FileCache::getCommonUser(), read_buffer_creator, - read_settings, + modified_read_settings, std::string(CurrentThread::getQueryId()), object_size, - /* allow_seeks */false, + /* allow_seeks */true, /* use_external_buffer */true, /* read_until_position */std::nullopt, context_->getFilesystemCacheLog()); @@ -494,16 +510,25 @@ std::unique_ptr StorageObjectStorageSource::createReadBu } } + if (!impl) + impl = object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), modified_read_settings); + + if (!use_async_buffer) + return impl; + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); impl = std::make_unique( - std::move(impl), reader, read_settings, + std::move(impl), reader, modified_read_settings, context_->getAsyncReadCounters(), context_->getFilesystemReadPrefetchesLog()); - impl->setReadUntilEnd(); - impl->prefetch(DEFAULT_PREFETCH_PRIORITY); + if (use_prefetch) + { + impl->setReadUntilEnd(); + impl->prefetch(DEFAULT_PREFETCH_PRIORITY); + } return impl; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index cd5bcbb32ff..f313b1bf3c3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -67,7 +67,7 @@ public: bool include_connection_info = true); static std::unique_ptr createReadBuffer( - const ObjectInfo & object_info, + ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log); diff --git a/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml b/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml new file mode 100644 index 00000000000..e91362640fe --- /dev/null +++ b/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml @@ -0,0 +1,8 @@ + + + + 1Gi + cache1 + + + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b2306d487ec..9679ee7736f 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -68,7 +68,10 @@ def started_cluster(): cluster = ClickHouseCluster(__file__, with_spark=True) cluster.add_instance( "node1", - main_configs=["configs/config.d/named_collections.xml"], + main_configs=[ + "configs/config.d/named_collections.xml", + "configs/config.d/filesystem_caches.xml", + ], user_configs=["configs/users.d/users.xml"], with_minio=True, with_azurite=True, @@ -829,3 +832,56 @@ def test_restart_broken_s3(started_cluster): ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + +@pytest.mark.parametrize("storage_type", ["s3"]) +def test_filesystem_cache(started_cluster, storage_type): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + TABLE_NAME = "test_filesystem_cache_" + storage_type + "_" + get_uuid_str() + + write_iceberg_from_df( + spark, + generate_data(spark, 0, 10), + TABLE_NAME, + mode="overwrite", + format_version="1", + partition_by="a", + ) + + default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + ) + + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) + + query_id = f"{TABLE_NAME}-{uuid.uuid4()}" + instance.query( + f"SELECT * FROM {TABLE_NAME} SETTINGS filesystem_cache_name = 'cache1'", + query_id=query_id, + ) + + instance.query("SYSTEM FLUSH LOGS") + + count = int( + instance.query( + f"SELECT ProfileEvents['CachedReadBufferCacheWriteBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) + + query_id = f"{TABLE_NAME}-{uuid.uuid4()}" + instance.query( + f"SELECT * FROM {TABLE_NAME} SETTINGS filesystem_cache_name = 'cache1'", + query_id=query_id, + ) + + instance.query("SYSTEM FLUSH LOGS") + + assert count == int( + instance.query( + f"SELECT ProfileEvents['CachedReadBufferReadFromCacheBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) From 2777a0390ba98d1c5b459229e8f413852d4fb843 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Oct 2024 18:01:15 +0200 Subject: [PATCH 09/53] Add test for delta --- .../configs/config.d/filesystem_caches.xml | 8 +++ tests/integration/test_storage_delta/test.py | 67 ++++++++++++++++++- .../integration/test_storage_iceberg/test.py | 10 +++ tests/integration/test_storage_s3/test.py | 17 ++++- 4 files changed, 100 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_storage_delta/configs/config.d/filesystem_caches.xml diff --git a/tests/integration/test_storage_delta/configs/config.d/filesystem_caches.xml b/tests/integration/test_storage_delta/configs/config.d/filesystem_caches.xml new file mode 100644 index 00000000000..e91362640fe --- /dev/null +++ b/tests/integration/test_storage_delta/configs/config.d/filesystem_caches.xml @@ -0,0 +1,8 @@ + + + + 1Gi + cache1 + + + diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index c95193cc765..b18222c4555 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -12,6 +12,7 @@ import pyarrow as pa import pyarrow.parquet as pq import pyspark import pytest +import uuid from delta import * from deltalake.writer import write_deltalake from minio.deleteobjects import DeleteObject @@ -70,7 +71,10 @@ def started_cluster(): cluster = ClickHouseCluster(__file__, with_spark=True) cluster.add_instance( "node1", - main_configs=["configs/config.d/named_collections.xml"], + main_configs=[ + "configs/config.d/named_collections.xml", + "configs/config.d/filesystem_caches.xml", + ], user_configs=["configs/users.d/users.xml"], with_minio=True, stay_alive=True, @@ -826,3 +830,64 @@ def test_complex_types(started_cluster): f"SELECT metadata FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/root/{table_name}' , 'minio', 'minio123')" ) ) + + +@pytest.mark.parametrize("storage_type", ["s3"]) +def test_filesystem_cache(started_cluster, storage_type): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + TABLE_NAME = randomize_table_name("test_filesystem_cache") + bucket = started_cluster.minio_bucket + + if not minio_client.bucket_exists(bucket): + minio_client.make_bucket(bucket) + + parquet_data_path = create_initial_data_file( + started_cluster, + instance, + "SELECT number, toString(number) FROM numbers(100)", + TABLE_NAME, + ) + + write_delta_from_file(spark, parquet_data_path, f"/{TABLE_NAME}") + upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") + create_delta_table(instance, TABLE_NAME, bucket=bucket) + + query_id = f"{TABLE_NAME}-{uuid.uuid4()}" + instance.query( + f"SELECT * FROM {TABLE_NAME} SETTINGS filesystem_cache_name = 'cache1'", + query_id=query_id, + ) + + instance.query("SYSTEM FLUSH LOGS") + + count = int( + instance.query( + f"SELECT ProfileEvents['CachedReadBufferCacheWriteBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) + assert 0 < int( + instance.query( + f"SELECT ProfileEvents['S3GetObject'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) + + query_id = f"{TABLE_NAME}-{uuid.uuid4()}" + instance.query( + f"SELECT * FROM {TABLE_NAME} SETTINGS filesystem_cache_name = 'cache1'", + query_id=query_id, + ) + + instance.query("SYSTEM FLUSH LOGS") + + assert count == int( + instance.query( + f"SELECT ProfileEvents['CachedReadBufferReadFromCacheBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) + assert 0 == int( + instance.query( + f"SELECT ProfileEvents['S3GetObject'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 9679ee7736f..a0344141b0d 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -871,6 +871,11 @@ def test_filesystem_cache(started_cluster, storage_type): f"SELECT ProfileEvents['CachedReadBufferCacheWriteBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" ) ) + assert 0 < int( + instance.query( + f"SELECT ProfileEvents['S3GetObject'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) query_id = f"{TABLE_NAME}-{uuid.uuid4()}" instance.query( @@ -885,3 +890,8 @@ def test_filesystem_cache(started_cluster, storage_type): f"SELECT ProfileEvents['CachedReadBufferReadFromCacheBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" ) ) + assert 0 == int( + instance.query( + f"SELECT ProfileEvents['S3GetObject'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index c721e2ff7b3..cdf7016917c 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2422,6 +2422,11 @@ def test_filesystem_cache(started_cluster): ) assert count == 290 + assert 0 < int( + instance.query( + f"SELECT ProfileEvents['S3GetObject'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) query_id = f"{table_name}-{uuid.uuid4()}" instance.query( @@ -2431,8 +2436,18 @@ def test_filesystem_cache(started_cluster): instance.query("SYSTEM FLUSH LOGS") - assert count == int( + assert count * 2 == int( instance.query( f"SELECT ProfileEvents['CachedReadBufferReadFromCacheBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" ) ) + assert 0 == int( + instance.query( + f"SELECT ProfileEvents['CachedReadBufferCacheWriteBytes'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) + assert 0 == int( + instance.query( + f"SELECT ProfileEvents['S3GetObject'] FROM system.query_log WHERE query_id = '{query_id}' AND type = 'QueryFinish'" + ) + ) From 73ede0201f83b193beec16e2d8fee1a5281cf653 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Oct 2024 16:09:20 +0000 Subject: [PATCH 10/53] Automatic style fix --- tests/integration/test_storage_delta/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index b18222c4555..f2ac9026b88 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -5,6 +5,7 @@ import os import random import string import time +import uuid from datetime import datetime import delta @@ -12,7 +13,6 @@ import pyarrow as pa import pyarrow.parquet as pq import pyspark import pytest -import uuid from delta import * from deltalake.writer import write_deltalake from minio.deleteobjects import DeleteObject From 383559fbbeef4d9746e0b8c39f00ca1fd3bc7753 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Oct 2024 20:16:47 +0200 Subject: [PATCH 11/53] Fix test --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f35f303b65f..220ee13cb25 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1412,7 +1412,7 @@ def test_parallel_read(cluster): res = azure_query( node, - f"select count() from azureBlobStorage('{connection_string}', 'cont', 'test_parallel_read.parquet')", + f"select count() from azureBlobStorage('{connection_string}', 'cont', 'test_parallel_read.parquet') settings remote_filesystem_read_method='read'", ) assert int(res) == 10000 assert_logs_contain_with_retry(node, "AzureBlobStorage readBigAt read bytes") From bd4c3bafcb63dd0bac18dc8de29ac6851c1b7686 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Oct 2024 16:45:39 +0000 Subject: [PATCH 12/53] Rename enable_secure_identifiers -> enforce_strict_identifier_format Signed-off-by: vdimir --- docs/en/operations/settings/settings.md | 4 +-- src/Core/Settings.cpp | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/executeQuery.cpp | 10 +++--- src/Parsers/IAST.cpp | 9 ++--- src/Parsers/IAST.h | 8 ++--- .../03234_enable_secure_identifiers.sql | 34 +++++++++---------- 7 files changed, 35 insertions(+), 34 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 167dec75749..f3c1f6ef283 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5683,9 +5683,9 @@ Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting Default value: `false`. -## enable_secure_identifiers +## enforce_strict_identifier_format -If enabled, only allow secure identifiers which contain only underscore and alphanumeric characters +If enabled, only allow identifiers containing alphanumeric characters and underscores. Default value: `false`. diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 0e0e3ddec6e..2d3927c8026 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -920,7 +920,7 @@ namespace ErrorCodes M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ M(Bool, restore_replace_external_dictionary_source_to_null, false, "Replace external dictionary sources to Null on restore. Useful for testing purposes", 0) \ M(Bool, create_if_not_exists, false, "Enable IF NOT EXISTS for CREATE statements by default", 0) \ - M(Bool, enable_secure_identifiers, false, "If enabled, only allow secure identifiers which contain only underscore and alphanumeric characters", 0) \ + M(Bool, enforce_strict_identifier_format, false, "If enabled, only allow identifiers containing alphanumeric characters and underscores", 0) \ M(Bool, mongodb_throw_on_unsupported_query, true, "If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'.", 0) \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index beeaad5b5d3..7bf7f04c679 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -74,7 +74,7 @@ static std::initializer_list executeQueryImpl( InterpreterSetQuery::applySettingsFromQuery(ast, context); validateAnalyzerSettings(ast, settings[Setting::allow_experimental_analyzer]); - if (settings[Setting::enable_secure_identifiers]) + if (settings[Setting::enforce_strict_identifier_format]) { WriteBufferFromOwnString buf; - IAST::FormatSettings enable_secure_identifiers_settings(buf, true); - enable_secure_identifiers_settings.enable_secure_identifiers = true; - ast->format(enable_secure_identifiers_settings); + IAST::FormatSettings enforce_strict_identifier_format_settings(buf, true); + enforce_strict_identifier_format_settings.enforce_strict_identifier_format = true; + ast->format(enforce_strict_identifier_format_settings); } if (auto * insert_query = ast->as()) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 8aac0c4ce4f..2058c7c60cf 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include namespace DB @@ -265,14 +266,14 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool ambiguous) void IAST::FormatSettings::checkIdentifier(const String & name) const { - if (enable_secure_identifiers) + if (enforce_strict_identifier_format) { - bool is_secure_identifier = std::all_of(name.begin(), name.end(), [](char ch) { return std::isalnum(ch) || ch == '_'; }); - if (!is_secure_identifier) + bool is_word_char_identifier = std::all_of(name.begin(), name.end(), isWordCharASCII); + if (!is_word_char_identifier) { throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Not a secure identifier: `{}`, a secure identifier must contain only underscore and alphanumeric characters", + "Identifier '{}' contains characters other than alphanumeric and cannot be when enforce_strict_identifier_format is enabled", name); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 8e307e0383b..eb6ddcc5d8b 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -202,7 +202,7 @@ public: char nl_or_ws; /// Newline or whitespace. LiteralEscapingStyle literal_escaping_style; bool print_pretty_type_names; - bool enable_secure_identifiers; + bool enforce_strict_identifier_format; explicit FormatSettings( WriteBuffer & ostr_, @@ -213,7 +213,7 @@ public: bool show_secrets_ = true, LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular, bool print_pretty_type_names_ = false, - bool enable_secure_identifiers_ = false) + bool enforce_strict_identifier_format_ = false) : ostr(ostr_) , one_line(one_line_) , hilite(hilite_) @@ -223,7 +223,7 @@ public: , nl_or_ws(one_line ? ' ' : '\n') , literal_escaping_style(literal_escaping_style_) , print_pretty_type_names(print_pretty_type_names_) - , enable_secure_identifiers(enable_secure_identifiers_) + , enforce_strict_identifier_format(enforce_strict_identifier_format_) { } @@ -237,7 +237,7 @@ public: , nl_or_ws(other.nl_or_ws) , literal_escaping_style(other.literal_escaping_style) , print_pretty_type_names(other.print_pretty_type_names) - , enable_secure_identifiers(other.enable_secure_identifiers) + , enforce_strict_identifier_format(other.enforce_strict_identifier_format) { } diff --git a/tests/queries/0_stateless/03234_enable_secure_identifiers.sql b/tests/queries/0_stateless/03234_enable_secure_identifiers.sql index dfeb53dd147..7381ffb78ad 100644 --- a/tests/queries/0_stateless/03234_enable_secure_identifiers.sql +++ b/tests/queries/0_stateless/03234_enable_secure_identifiers.sql @@ -8,7 +8,7 @@ PRIMARY KEY (town, date) PARTITION BY toYear(date) COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level SETTINGS - enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + enforce_strict_identifier_format=true; -- { serverError BAD_ARGUMENTS } DROP TABLE IF EXISTS `test_foo_#`; @@ -23,7 +23,7 @@ PRIMARY KEY (town, date) PARTITION BY toYear(date) COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level SETTINGS - enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + enforce_strict_identifier_format=true; -- { serverError BAD_ARGUMENTS } DROP TABLE IF EXISTS test_foo; CREATE TABLE test_foo ( @@ -36,7 +36,7 @@ PRIMARY KEY (town, date) PARTITION BY toYear(date) COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level SETTINGS - enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + enforce_strict_identifier_format=true; -- { serverError BAD_ARGUMENTS } DROP TABLE IF EXISTS test_foo; CREATE TABLE test_foo ( @@ -49,7 +49,7 @@ PRIMARY KEY (town, date) PARTITION BY toYear(date) COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level SETTINGS - enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + enforce_strict_identifier_format=true; -- { serverError BAD_ARGUMENTS } DROP TABLE IF EXISTS test_foo; CREATE TABLE test_foo ( @@ -62,11 +62,11 @@ PRIMARY KEY (town, date) PARTITION BY toYear(date) COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level SETTINGS - enable_secure_identifiers=true; + enforce_strict_identifier_format=true; -SHOW CREATE TABLE test_foo +SHOW CREATE TABLE test_foo SETTINGS - enable_secure_identifiers=true; + enforce_strict_identifier_format=true; DROP TABLE IF EXISTS test_foo; CREATE TABLE test_foo ( @@ -79,13 +79,13 @@ PRIMARY KEY (town, date) PARTITION BY toYear(date) COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level SETTINGS - enable_secure_identifiers=true; + enforce_strict_identifier_format=true; -SHOW CREATE TABLE test_foo +SHOW CREATE TABLE test_foo SETTINGS - enable_secure_identifiers=true; + enforce_strict_identifier_format=true; --- CREATE TABLE without `enable_secure_identifiers` +-- CREATE TABLE without `enforce_strict_identifier_format` DROP TABLE IF EXISTS test_foo; CREATE TABLE `test_foo` ( `insecure_$` Int8, @@ -95,17 +95,17 @@ CREATE TABLE `test_foo` ( ENGINE = MergeTree PRIMARY KEY (town, date) PARTITION BY toYear(date); --- Then SHOW CREATE .. with `enable_secure_identifiers` +-- Then SHOW CREATE .. with `enforce_strict_identifier_format` -- While the result contains insecure identifiers (`insecure_$`), the `SHOW CREATE TABLE ...` query does not have any. So the query is expected to succeed. -SHOW CREATE TABLE test_foo +SHOW CREATE TABLE test_foo SETTINGS - enable_secure_identifiers=true; + enforce_strict_identifier_format=true; DROP TABLE IF EXISTS test_foo; --- SHOW CREATE .. query contains an insecure identifier (`test_foo$`) with `enable_secure_identifiers` +-- SHOW CREATE .. query contains an insecure identifier (`test_foo$`) with `enforce_strict_identifier_format` SHOW CREATE TABLE `test_foo$` SETTINGS - enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + enforce_strict_identifier_format=true; -- { serverError BAD_ARGUMENTS } -DROP TABLE IF EXISTS test_foo; \ No newline at end of file +DROP TABLE IF EXISTS test_foo; From cc6b461115bcee0f4c26464e44c5d8825c8717dc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 4 Oct 2024 10:21:25 +0200 Subject: [PATCH 13/53] Fix possible use-after-free in SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf The problem is that getMessageTypeForFormatSchema() returns raw pointer (google::protobuf::Descriptor *), and ProtobufSchemas::clear() will free all this pointers, fix this by storing shared_ptr in cache. Signed-off-by: Azat Khuzhin --- src/Formats/ProtobufSchemas.cpp | 6 ++--- src/Formats/ProtobufSchemas.h | 27 +++++++++++++++---- src/Formats/ProtobufSerializer.cpp | 14 +++++++--- src/Formats/ProtobufSerializer.h | 7 ++--- .../Formats/Impl/ProtobufListInputFormat.cpp | 8 +++--- .../Formats/Impl/ProtobufListOutputFormat.cpp | 2 +- .../Formats/Impl/ProtobufRowInputFormat.cpp | 10 +++---- .../Formats/Impl/ProtobufRowInputFormat.h | 4 +-- .../Formats/Impl/ProtobufRowOutputFormat.cpp | 2 +- 9 files changed, 52 insertions(+), 28 deletions(-) diff --git a/src/Formats/ProtobufSchemas.cpp b/src/Formats/ProtobufSchemas.cpp index 1e355d0484f..2b6cffbbaa1 100644 --- a/src/Formats/ProtobufSchemas.cpp +++ b/src/Formats/ProtobufSchemas.cpp @@ -112,7 +112,7 @@ private: }; -const google::protobuf::Descriptor * +ProtobufSchemas::DescriptorHolder ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope, const String & google_protos_path) { std::lock_guard lock(mutex); @@ -121,10 +121,10 @@ ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo & info, Wi it = importers .emplace( info.schemaDirectory(), - std::make_unique(info.schemaDirectory(), google_protos_path, with_envelope)) + std::make_shared(info.schemaDirectory(), google_protos_path, with_envelope)) .first; auto * importer = it->second.get(); - return importer->import(info.schemaPath(), info.messageName()); + return DescriptorHolder(it->second, importer->import(info.schemaPath(), info.messageName())); } } diff --git a/src/Formats/ProtobufSchemas.h b/src/Formats/ProtobufSchemas.h index 066f89d5176..378d53eb489 100644 --- a/src/Formats/ProtobufSchemas.h +++ b/src/Formats/ProtobufSchemas.h @@ -57,14 +57,31 @@ public: // Clear cached protobuf schemas void clear(); - /// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type. - /// The function never returns nullptr, it throws an exception if it cannot load or parse the file. - const google::protobuf::Descriptor * + class ImporterWithSourceTree; + struct DescriptorHolder + { + DescriptorHolder(std::shared_ptr importer_, const google::protobuf::Descriptor * message_descriptor_) + : importer(std::move(importer_)) + , message_descriptor(message_descriptor_) + {} + private: + std::shared_ptr importer; + public: + const google::protobuf::Descriptor * message_descriptor; + }; + + /// Parses the format schema, then parses the corresponding proto file, and + /// returns holder (since the descriptor only valid if + /// ImporterWithSourceTree is valid): + /// + /// {ImporterWithSourceTree, protobuf::Descriptor - descriptor of the message type}. + /// + /// The function always return valid message descriptor, it throws an exception if it cannot load or parse the file. + DescriptorHolder getMessageTypeForFormatSchema(const FormatSchemaInfo & info, WithEnvelope with_envelope, const String & google_protos_path); private: - class ImporterWithSourceTree; - std::unordered_map> importers; + std::unordered_map> importers; std::mutex mutex; }; diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 20426e8eccb..e873eaabdf4 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3871,26 +3871,32 @@ std::unique_ptr ProtobufSerializer::create( const Strings & column_names, const DataTypes & data_types, std::vector & missing_column_indices, - const google::protobuf::Descriptor & message_descriptor, + const ProtobufSchemas::DescriptorHolder & descriptor, bool with_length_delimiter, bool with_envelope, bool flatten_google_wrappers, ProtobufReader & reader) { - return ProtobufSerializerBuilder(reader).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope, flatten_google_wrappers); + return ProtobufSerializerBuilder(reader).buildMessageSerializer( + column_names, data_types, missing_column_indices, + *descriptor.message_descriptor, + with_length_delimiter, with_envelope, flatten_google_wrappers); } std::unique_ptr ProtobufSerializer::create( const Strings & column_names, const DataTypes & data_types, - const google::protobuf::Descriptor & message_descriptor, + const ProtobufSchemas::DescriptorHolder & descriptor, bool with_length_delimiter, bool with_envelope, bool defaults_for_nullable_google_wrappers, ProtobufWriter & writer) { std::vector missing_column_indices; - return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter, with_envelope, defaults_for_nullable_google_wrappers); + return ProtobufSerializerBuilder(writer).buildMessageSerializer( + column_names, data_types, missing_column_indices, + *descriptor.message_descriptor, + with_length_delimiter, with_envelope, defaults_for_nullable_google_wrappers); } NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor, bool skip_unsupported_fields) diff --git a/src/Formats/ProtobufSerializer.h b/src/Formats/ProtobufSerializer.h index d50f7e4956e..f40deb0c6d1 100644 --- a/src/Formats/ProtobufSerializer.h +++ b/src/Formats/ProtobufSerializer.h @@ -4,7 +4,8 @@ #if USE_PROTOBUF # include -#include +# include +# include namespace google::protobuf { class Descriptor; } @@ -39,7 +40,7 @@ public: const Strings & column_names, const DataTypes & data_types, std::vector & missing_column_indices, - const google::protobuf::Descriptor & message_descriptor, + const ProtobufSchemas::DescriptorHolder & descriptor, bool with_length_delimiter, bool with_envelope, bool flatten_google_wrappers, @@ -48,7 +49,7 @@ public: static std::unique_ptr create( const Strings & column_names, const DataTypes & data_types, - const google::protobuf::Descriptor & message_descriptor, + const ProtobufSchemas::DescriptorHolder & descriptor, bool with_length_delimiter, bool with_envelope, bool defaults_for_nullable_google_wrappers, diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp index c643ae060d6..2f3c9b6c6db 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp @@ -23,7 +23,7 @@ ProtobufListInputFormat::ProtobufListInputFormat( header_.getNames(), header_.getDataTypes(), missing_column_indices, - *ProtobufSchemas::instance().getMessageTypeForFormatSchema( + ProtobufSchemas::instance().getMessageTypeForFormatSchema( schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes, google_protos_path), /* with_length_delimiter = */ true, /* with_envelope = */ true, @@ -93,9 +93,9 @@ ProtobufListSchemaReader::ProtobufListSchemaReader(const FormatSettings & format NamesAndTypesList ProtobufListSchemaReader::readSchema() { - const auto * message_descriptor - = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::Yes, google_protos_path); - return protobufSchemaToCHSchema(message_descriptor, skip_unsupported_fields); + auto descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema( + schema_info, ProtobufSchemas::WithEnvelope::Yes, google_protos_path); + return protobufSchemaToCHSchema(descriptor.message_descriptor, skip_unsupported_fields); } void registerInputFormatProtobufList(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp index 927301fb1b0..8a7d52b9c28 100644 --- a/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp @@ -20,7 +20,7 @@ ProtobufListOutputFormat::ProtobufListOutputFormat( , serializer(ProtobufSerializer::create( header_.getNames(), header_.getDataTypes(), - *ProtobufSchemas::instance().getMessageTypeForFormatSchema( + ProtobufSchemas::instance().getMessageTypeForFormatSchema( schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes, google_protos_path), /* with_length_delimiter = */ true, /* with_envelope = */ true, diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index 7340faf785c..57d4f65e74e 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -19,7 +19,7 @@ ProtobufRowInputFormat::ProtobufRowInputFormat( bool flatten_google_wrappers_, const String & google_protos_path) : IRowInputFormat(header_, in_, params_) - , message_descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema( + , descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema( schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No, google_protos_path)) , with_length_delimiter(with_length_delimiter_) , flatten_google_wrappers(flatten_google_wrappers_) @@ -33,7 +33,7 @@ void ProtobufRowInputFormat::createReaderAndSerializer() getPort().getHeader().getNames(), getPort().getHeader().getDataTypes(), missing_column_indices, - *message_descriptor, + descriptor, with_length_delimiter, /* with_envelope = */ false, flatten_google_wrappers, @@ -132,9 +132,9 @@ ProtobufSchemaReader::ProtobufSchemaReader(const FormatSettings & format_setting NamesAndTypesList ProtobufSchemaReader::readSchema() { - const auto * message_descriptor - = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info, ProtobufSchemas::WithEnvelope::No, google_protos_path); - return protobufSchemaToCHSchema(message_descriptor, skip_unsupported_fields); + auto descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema( + schema_info, ProtobufSchemas::WithEnvelope::No, google_protos_path); + return protobufSchemaToCHSchema(descriptor.message_descriptor, skip_unsupported_fields); } void registerProtobufSchemaReader(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 10ce37f9087..93dbaf35bdd 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -6,7 +6,7 @@ # include # include # include -# include +# include namespace DB { @@ -57,7 +57,7 @@ private: std::vector missing_column_indices; std::unique_ptr serializer; - const google::protobuf::Descriptor * message_descriptor; + const ProtobufSchemas::DescriptorHolder descriptor; bool with_length_delimiter; bool flatten_google_wrappers; }; diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 1deec264a56..4a83aa2f921 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -27,7 +27,7 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat( , serializer(ProtobufSerializer::create( header_.getNames(), header_.getDataTypes(), - *ProtobufSchemas::instance().getMessageTypeForFormatSchema( + ProtobufSchemas::instance().getMessageTypeForFormatSchema( schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No, settings_.protobuf.google_protos_path), with_length_delimiter_, /* with_envelope = */ false, From 6358a1483456fb10e558a73ae059c3b3cac54839 Mon Sep 17 00:00:00 2001 From: Sharath K S Date: Sat, 5 Oct 2024 18:52:36 -0700 Subject: [PATCH 14/53] Handle fast test failure --- src/Core/Settings.cpp | 10 ++++++++++ src/Core/Settings.h | 1 + src/Interpreters/executeQuery.cpp | 6 +++--- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 07b4ecd7a24..405cce8e829 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1390,6 +1390,16 @@ std::vector Settings::getUnchangedNames() const return setting_names; } +std::vector Settings::getChangedNames() const +{ + std::vector setting_names; + for (const auto & setting : impl->allChanged()) + { + setting_names.emplace_back(setting.getName()); + } + return setting_names; +} + void Settings::dumpToSystemSettingsColumns(MutableColumnsAndConstraints & params) const { MutableColumns & res_columns = params.res_columns; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6bb66039afb..74870197de2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -131,6 +131,7 @@ struct Settings std::vector getAllRegisteredNames() const; std::vector getChangedAndObsoleteNames() const; std::vector getUnchangedNames() const; + std::vector getChangedNames() const; void dumpToSystemSettingsColumns(MutableColumnsAndConstraints & params) const; void dumpToMapColumn(IColumn * column, bool changed_only = true) const; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 87f15d0a859..ecc1e9479b9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -570,11 +570,11 @@ void logQueryFinish( query_span->addAttribute("clickhouse.user", user_name); } - if (settings.log_query_settings) + if (settings[Setting::log_query_settings]) { - for (const auto & setting : settings.allChanged()) + auto changed_settings_names = settings.getChangedNames(); + for (const auto & name : changed_settings_names) { - auto name = setting.getName(); Field value = settings.get(name); String value_str; From e701cec0f2ce92c17a146ba9e75ec72056b238fe Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 5 Oct 2024 22:46:09 +0200 Subject: [PATCH 15/53] Add a test for SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf I was able to reproduce the problem, but only with ASan build, but CI is better them me in this job. Signed-off-by: Azat Khuzhin --- ...FORMAT_SCHEMA_CACHE_FOR_Protobuf.reference | 0 ...M_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh | 41 +++++++++++++++++++ .../0_stateless/format_schemas/03250.proto | 5 +++ 3 files changed, 46 insertions(+) create mode 100644 tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.reference create mode 100755 tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh create mode 100644 tests/queries/0_stateless/format_schemas/03250.proto diff --git a/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.reference b/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh b/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh new file mode 100755 index 00000000000..3d6b59882de --- /dev/null +++ b/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash +# Tags: race + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SCHEMADIR=$CLICKHOUSE_SCHEMA_FILES +CLIENT_SCHEMADIR=$CURDIR/format_schemas +export SERVER_SCHEMADIR=$CLICKHOUSE_DATABASE +mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR +cp -r $CLIENT_SCHEMADIR/03250.proto $SCHEMADIR/$SERVER_SCHEMADIR/ + +$CLICKHOUSE_CLIENT --query "SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf" + +export BINARY_FILE_PATH +BINARY_FILE_PATH=$(mktemp "$CLICKHOUSE_USER_FILES/03250.XXXXXX.binary") +$CLICKHOUSE_CLIENT --query "SELECT * FROM numbers(10) FORMAT Protobuf SETTINGS format_schema = '$CLIENT_SCHEMADIR/03250:Numbers'" > $BINARY_FILE_PATH + +function protobuf_reader() +{ + while true; do + $CLICKHOUSE_CLIENT --query "SELECT count() FROM file('$(basename $BINARY_FILE_PATH)', 'Protobuf') FORMAT Null SETTINGS max_threads=1, format_schema='$SERVER_SCHEMADIR/03250:Numbers'" + done +} +export -f protobuf_reader + +function protobuf_cache_drainer() +{ + while true; do + $CLICKHOUSE_CLIENT --query "SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf" + done +} +export -f protobuf_cache_drainer + +timeout 20 bash -c protobuf_reader & +timeout 20 bash -c protobuf_cache_drainer & +wait + +rm -f "${BINARY_FILE_PATH:?}" +rm -fr "${SCHEMADIR:?}/${SERVER_SCHEMADIR:?}/" diff --git a/tests/queries/0_stateless/format_schemas/03250.proto b/tests/queries/0_stateless/format_schemas/03250.proto new file mode 100644 index 00000000000..aaeea4a386e --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/03250.proto @@ -0,0 +1,5 @@ +syntax = "proto3"; + +message Numbers { + uint64 number = 1; +}; From eca9ec566a63a9046bb3f1c9373d671946a95ba4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 6 Oct 2024 14:48:52 +0200 Subject: [PATCH 16/53] tests: fix permissions for files in user_files Signed-off-by: Azat Khuzhin --- .../03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh b/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh index 3d6b59882de..d027a77907c 100755 --- a/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh +++ b/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh @@ -16,6 +16,7 @@ $CLICKHOUSE_CLIENT --query "SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf" export BINARY_FILE_PATH BINARY_FILE_PATH=$(mktemp "$CLICKHOUSE_USER_FILES/03250.XXXXXX.binary") $CLICKHOUSE_CLIENT --query "SELECT * FROM numbers(10) FORMAT Protobuf SETTINGS format_schema = '$CLIENT_SCHEMADIR/03250:Numbers'" > $BINARY_FILE_PATH +chmod 666 "$BINARY_FILE_PATH" function protobuf_reader() { From 7b6aa982a3f4a2290b348b6333cd24d2be25f238 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Oct 2024 12:14:09 +0200 Subject: [PATCH 17/53] Update src/Core/Settings.cpp --- src/Core/Settings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 425e7c1ec2d..e3761e22def 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5498,7 +5498,7 @@ Replace external dictionary sources to Null on restore. Useful for testing purpo Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown. )", 0) \ M(Bool, enforce_strict_identifier_format, false, R"( -If enabled, only allow secure identifiers which contain only underscore and alphanumeric characters +If enabled, only allow identifiers containing alphanumeric characters and underscores. )", 0) \ M(Bool, mongodb_throw_on_unsupported_query, true, R"( If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'. From 250e8c6119a25f6d78d5c2c093fb294248f7ee20 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 8 Oct 2024 10:43:44 +0200 Subject: [PATCH 18/53] Fix setting description --- src/Core/Settings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 2d64a581359..d284d69ea76 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -4813,7 +4813,7 @@ Max attempts to read with backoff Use cache for remote filesystem. This setting does not turn on/off cache for disks (must be done via disk config), but allows to bypass cache for some queries if intended )", 0) \ M(String, filesystem_cache_name, "", R"( -Use cache for remote filesystem. This setting does not turn on/off cache for disks (must be done via disk config), but allows to bypass cache for some queries if intended +Filesystem cache name to use for stateless table engines or data lakes )", 0) \ M(Bool, enable_filesystem_cache_on_write_operations, false, R"( Write into cache on write operations. To actually work this setting requires be added to disk config too From 15a86aeaf6103d10b2cfdd1d114e1fa1feea0f5f Mon Sep 17 00:00:00 2001 From: Sharath K S Date: Tue, 8 Oct 2024 07:33:33 -0700 Subject: [PATCH 19/53] Add prefix "clickhouse.setting." to query setting names in span log Updated relevant documentation and setting descriptions to reflect this change. --- docs/en/operations/opentelemetry.md | 3 +++ docs/en/operations/settings/settings.md | 7 +++++++ src/Interpreters/executeQuery.cpp | 3 ++- .../0_stateless/02421_simple_queries_for_opentelemetry.sh | 6 +++--- 4 files changed, 15 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index fe60ceedc0b..576e4adb078 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -31,6 +31,9 @@ The table must be enabled in the server configuration, see the `opentelemetry_sp The tags or attributes are saved as two parallel arrays, containing the keys and values. Use [ARRAY JOIN](../sql-reference/statements/select/array-join.md) to work with them. +## Log-query-settings +ClickHouse allows you to log changes to query settings during query execution. When enabled, any modifications made to query settings will be recorded in the OpenTelemetry span log. This feature is particularly useful in production environments for tracking configuration changes that may affect query performance. + ## Integration with monitoring systems At the moment, there is no ready tool that can export the tracing data from ClickHouse to a monitoring system. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 392b1831ce3..b5ba6eef50b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -660,6 +660,13 @@ See also: - [GROUP BY clause](/docs/en/sql-reference/statements/select/group-by.md) +## log_query_settings {#log-query-settings} +Type: Bool + +Default value: 1 + +Log query settings into the query_log and opentelemetry_span_log. + ## partial_merge_join_optimizations {#partial_merge_join_optimizations} Disables optimizations in partial merge join algorithm for [JOIN](../../sql-reference/statements/select/join.md) queries. diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ecc1e9479b9..681eb64cc7e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -600,7 +600,8 @@ void logQueryFinish( break; } - query_span->addAttribute(name, value_str); + query_span->addAttribute(fmt::format("clickhouse.setting.{}", name), value_str); + } } query_span->finish(); diff --git a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh index e36c8c36cbf..59310b970d1 100755 --- a/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh +++ b/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.sh @@ -48,9 +48,9 @@ function check_query_settings() { result=$(${CLICKHOUSE_CLIENT} -q " SYSTEM FLUSH LOGS; - SELECT attribute['min_compress_block_size'], - attribute['max_block_size'], - attribute['max_execution_time'] + SELECT attribute['clickhouse.setting.min_compress_block_size'], + attribute['clickhouse.setting.max_block_size'], + attribute['clickhouse.setting.max_execution_time'] FROM system.opentelemetry_span_log WHERE finish_date >= yesterday() AND operation_name = 'query' From 6d308b31c09f9678529f6223369183dbabf9b851 Mon Sep 17 00:00:00 2001 From: Sharath K S Date: Tue, 8 Oct 2024 10:11:16 -0700 Subject: [PATCH 20/53] Updated log_query_settings description in Settings.cpp --- src/Core/Settings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 70ddc054665..5b7571b2362 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -2700,7 +2700,7 @@ The maximum read speed in bytes per second for particular backup on server. Zero Log query performance statistics into the query_log, query_thread_log and query_views_log. )", 0) \ M(Bool, log_query_settings, true, R"( -Log query settings into the query_log. +Log query settings into the query_log and OpenTelemetry span log. )", 0) \ M(Bool, log_query_threads, false, R"( Setting up query threads logging. From 6f3c1fe0e89bb1c3715e02bca5519baa9e88bc94 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 10 Oct 2024 11:55:46 +0800 Subject: [PATCH 21/53] Try fix test --- tests/integration/test_storage_s3/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index cdf7016917c..968b5cb5fe5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2435,6 +2435,7 @@ def test_filesystem_cache(started_cluster): ) instance.query("SYSTEM FLUSH LOGS") + instance.query("SYSTEM DROP SCHEMA CACHE") assert count * 2 == int( instance.query( From 367cdb72274659d520ae30685fa38f297e5406d3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Oct 2024 13:40:08 +0800 Subject: [PATCH 22/53] Add documentation --- .../integrations/azureBlobStorage.md | 27 ++++++++++++++++ .../table-engines/integrations/deltalake.md | 4 +++ .../table-engines/integrations/iceberg.md | 4 +++ .../engines/table-engines/integrations/s3.md | 32 +++++++++++++++++++ 4 files changed, 67 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index bdf96832e9d..bb1349ad9d0 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -63,7 +63,34 @@ Currently there are 3 ways to authenticate: - `SAS Token` - Can be used by providing an `endpoint`, `connection_string` or `storage_account_url`. It is identified by presence of '?' in the url. - `Workload Identity` - Can be used by providing an `endpoint` or `storage_account_url`. If `use_workload_identity` parameter is set in config, ([workload identity](https://github.com/Azure/azure-sdk-for-cpp/tree/main/sdk/identity/azure-identity#authenticate-azure-hosted-applications)) is used for authentication. +### Data cache {#data-cache} +`Azure` table engine supports data caching on local disk. +See filesystem cache configuration options and usage in this [section](/docs/en/operations/storing-data.md/#using-local-cache). +Caching is made depending on the path and ETag of the storage object, so clickhouse will not read a stale cache version. + +To enable caching use a setting `filesystem_cache_name = ''` and `enable_filesystem_cache = 1`. + +```sql +SELECT * +FROM azureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'test_container', 'test_table', 'CSV') +SETTINGS filesystem_cache_name = 'cache_for_azure', enable_filesystem_cache = 1; +``` + +1. add the following section to clickhouse configuration file: + +``` xml + + + + path to cache directory + 10Gi + + + +``` + +2. reuse cache configuration (and therefore cache storage) from clickhouse `storage_configuration` section, [described here](/docs/en/operations/storing-data.md/#using-local-cache) ## See also diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index 964c952f31a..fb564b4873e 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -48,6 +48,10 @@ Using named collections: CREATE TABLE deltalake ENGINE=DeltaLake(deltalake_conf, filename = 'test_table') ``` +### Data cache {#data-cache} + +`Iceberg` table engine and table function support data caching same as `S3`, `AzureBlobStorage`, `HDFS` storages. See [here](../../../engines/table-engines/integrations/s3.md#data-cache). + ## See also - [deltaLake table function](../../../sql-reference/table-functions/deltalake.md) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 94468066372..939312bafab 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -60,6 +60,10 @@ CREATE TABLE iceberg_table ENGINE=IcebergS3(iceberg_conf, filename = 'test_table Table engine `Iceberg` is an alias to `IcebergS3` now. +### Data cache {#data-cache} + +`Iceberg` table engine and table function support data caching same as `S3`, `AzureBlobStorage`, `HDFS` storages. See [here](../../../engines/table-engines/integrations/s3.md#data-cache). + ## See also - [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index f02d0563491..fb759b948a5 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -26,6 +26,7 @@ SELECT * FROM s3_engine_table LIMIT 2; │ two │ 2 │ └──────┴───────┘ ``` + ## Create Table {#creating-a-table} ``` sql @@ -43,6 +44,37 @@ CREATE TABLE s3_engine_table (name String, value UInt32) - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). - `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will auto-detect compression by file extension. +### Data cache {#data-cache} + +`S3` table engine supports data caching on local disk. +See filesystem cache configuration options and usage in this [section](/docs/en/operations/storing-data.md/#using-local-cache). +Caching is made depending on the path and ETag of the storage object, so clickhouse will not read a stale cache version. + +To enable caching use a setting `filesystem_cache_name = ''` and `enable_filesystem_cache = 1`. + +```sql +SELECT * +FROM s3('http://minio:10000/clickhouse//test_3.csv', 'minioadmin', 'minioadminpassword', 'CSV') +SETTINGS filesystem_cache_name = 'cache_for_s3', enable_filesystem_cache = 1; +``` + +There are two ways to define cache in configuration file. + +1. add the following section to clickhouse configuration file: + +``` xml + + + + path to cache directory + 10Gi + + + +``` + +2. reuse cache configuration (and therefore cache storage) from clickhouse `storage_configuration` section, [described here](/docs/en/operations/storing-data.md/#using-local-cache) + ### PARTITION BY `PARTITION BY` — Optional. In most cases you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). From 086196b4becf687874885734284d78e5a253c4aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Oct 2024 20:16:04 +0800 Subject: [PATCH 23/53] Fix typos --- .github/ISSUE_TEMPLATE/20_feature-request.md | 2 +- src/Processors/IProcessor.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/20_feature-request.md b/.github/ISSUE_TEMPLATE/20_feature-request.md index cf5ac000a23..1e5ccdbb3b3 100644 --- a/.github/ISSUE_TEMPLATE/20_feature-request.md +++ b/.github/ISSUE_TEMPLATE/20_feature-request.md @@ -15,7 +15,7 @@ assignees: '' **Use case** -> A clear and concise description of what is the intended usage scenario is. +> A clear and concise description of what the intended usage scenario is. **Describe the solution you'd like** diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index fd75eb530aa..ecd0cc6d4c4 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -365,11 +365,11 @@ public: /// Set limits for current storage. /// Different limits may be applied to different storages, we need to keep it per processor. - /// This method is need to be override only for sources. + /// This method is need to be overridden only for sources. virtual void setStorageLimits(const std::shared_ptr & /*storage_limits*/) {} /// This method is called for every processor without input ports. - /// Processor can return a new progress for the last read operation. + /// Processor can return new progress for the last read operation. /// You should zero internal counters in the call, in order to make in idempotent. virtual std::optional getReadProgress() { return std::nullopt; } From a0e2e6aadf6f4d1d541717f00000c7f037ef3e7b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 11 Oct 2024 14:36:02 +0200 Subject: [PATCH 24/53] Update docs/en/operations/settings/settings.md --- docs/en/operations/settings/settings.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3d46c45da81..f84af70de06 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -541,6 +541,7 @@ Default value: 0 Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*() ## log_query_settings {#log-query-settings} + Type: Bool Default value: 1 From 36e29e467cd34dd1de43e70a42e567b4aa5be80d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 11 Oct 2024 14:36:09 +0200 Subject: [PATCH 25/53] Update docs/en/operations/opentelemetry.md --- docs/en/operations/opentelemetry.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index 576e4adb078..48078197309 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -32,6 +32,7 @@ The table must be enabled in the server configuration, see the `opentelemetry_sp The tags or attributes are saved as two parallel arrays, containing the keys and values. Use [ARRAY JOIN](../sql-reference/statements/select/array-join.md) to work with them. ## Log-query-settings + ClickHouse allows you to log changes to query settings during query execution. When enabled, any modifications made to query settings will be recorded in the OpenTelemetry span log. This feature is particularly useful in production environments for tracking configuration changes that may affect query performance. ## Integration with monitoring systems From f832cc3dc9af4587a5100b53706f2044296cf9b3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 11 Oct 2024 15:01:02 +0200 Subject: [PATCH 26/53] Try fix test --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 968b5cb5fe5..b2513c900a0 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2401,7 +2401,7 @@ def test_filesystem_cache(started_cluster): id = uuid.uuid4() bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] - table_name = "test_filesystem_cache" + table_name = f"test_filesystem_cache-{uuid.uuid4()}" instance.query( f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{table_name}.tsv', auto, 'x UInt64') select number from numbers(100) SETTINGS s3_truncate_on_insert=1" From ab4f5f269e0e38e6d29c2bcf345932653637790c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Oct 2024 19:03:05 +0800 Subject: [PATCH 27/53] Update src/Processors/IProcessor.h Co-authored-by: Konstantin Bogdanov --- src/Processors/IProcessor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index ecd0cc6d4c4..01016299b9c 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -365,7 +365,7 @@ public: /// Set limits for current storage. /// Different limits may be applied to different storages, we need to keep it per processor. - /// This method is need to be overridden only for sources. + /// This method needs to be overridden only for sources. virtual void setStorageLimits(const std::shared_ptr & /*storage_limits*/) {} /// This method is called for every processor without input ports. From e58ed73d710b0de3c47477022f10fe23dbecc310 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sun, 13 Oct 2024 15:43:49 +0200 Subject: [PATCH 28/53] Update type-conversion-functions.md fix incorrect example --- .../en/sql-reference/functions/type-conversion-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index b37bd057adc..5c39f880a0e 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -5261,9 +5261,9 @@ SELECT toFixedString('foo', 8) AS s; Result: ```response -┌─s─────────────┬─s_cut─┐ -│ foo\0\0\0\0\0 │ foo │ -└───────────────┴───────┘ +┌─s─────────────┐ +│ foo\0\0\0\0\0 │ +└───────────────┘ ``` ## toStringCutToZero From 619f60b5ac17238d97170b0711babe071f3fcc38 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 14 Oct 2024 10:42:32 +0200 Subject: [PATCH 29/53] Fix test --- tests/integration/test_storage_s3/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index b2513c900a0..ad1842f4509 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2428,6 +2428,8 @@ def test_filesystem_cache(started_cluster): ) ) + instance.query("SYSTEM DROP SCHEMA CACHE") + query_id = f"{table_name}-{uuid.uuid4()}" instance.query( f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{table_name}.tsv') SETTINGS filesystem_cache_name = 'cache1', enable_filesystem_cache=1", @@ -2435,7 +2437,6 @@ def test_filesystem_cache(started_cluster): ) instance.query("SYSTEM FLUSH LOGS") - instance.query("SYSTEM DROP SCHEMA CACHE") assert count * 2 == int( instance.query( From d9bcc6639e82cc8a02ceb5a8eae1c3a60396e6f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 11:39:44 +0200 Subject: [PATCH 30/53] Increase max_rows_to_read in test reading from text_log --- tests/clickhouse-test | 1 + .../00002_log_and_exception_messages_formatting.sql | 2 +- tests/queries/0_stateless/00956_sensitive_data_masking.sh | 3 +-- tests/queries/0_stateless/00974_text_log_table_not_empty.sh | 4 ++-- .../queries/0_stateless/01165_lost_part_empty_partition.sql | 2 ++ .../0_stateless/01319_query_formatting_in_server_log.sql | 1 + .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 2 +- .../queries/0_stateless/01666_merge_tree_max_query_limit.sh | 2 +- tests/queries/0_stateless/02262_column_ttl.sh | 2 +- .../queries/0_stateless/02432_s3_parallel_parts_cleanup.sql | 1 + .../0_stateless/02434_cancel_insert_when_client_dies.sh | 2 +- .../02477_projection_materialize_and_zero_copy.sql | 1 + .../queries/0_stateless/02531_two_level_aggregation_bug.sh | 2 +- .../queries/0_stateless/02570_fallback_from_async_insert.sh | 1 + ...share_big_sets_between_multiple_mutations_tasks_long.sql | 1 + .../02581_share_big_sets_between_mutation_tasks_long.sql | 1 + .../0_stateless/02675_sparse_columns_clear_column.sql | 2 +- .../0_stateless/02796_calculate_text_stack_trace.sql | 1 + tests/queries/0_stateless/02813_starting_in_text_log.sql | 1 + .../02875_parallel_replicas_cluster_all_replicas.sql | 1 + .../queries/0_stateless/02875_parallel_replicas_remote.sql | 1 + .../02882_replicated_fetch_checksums_doesnt_match.sql | 1 + .../0_stateless/02898_parallel_replicas_progress_bar.sql | 1 + .../0_stateless/02903_rmt_retriable_merge_exception.sh | 6 ++++-- .../0_stateless/02935_parallel_replicas_settings.sql | 2 +- .../03096_text_log_format_string_args_not_empty.sql | 1 + tests/queries/0_stateless/03141_fetches_errors_stress.sql | 1 + .../0_stateless/03166_skip_indexes_vertical_merge_1.sql | 2 +- .../0_stateless/03166_skip_indexes_vertical_merge_2.sql | 1 + tests/queries/1_stateful/00177_memory_bound_merging.sh | 3 ++- 30 files changed, 36 insertions(+), 16 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 8fddebb8217..fdb5dd82561 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -292,6 +292,7 @@ def clickhouse_execute_http( "http_receive_timeout": timeout, "http_send_timeout": timeout, "output_format_parallel_formatting": 0, + "max_rows_to_read": 0 # Some queries read from system.text_log which might get too big } if settings is not None: params.update(settings) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 0f13217c236..917789aec10 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -9,7 +9,7 @@ system flush logs; drop table if exists logs; create view logs as select * from system.text_log where now() - toIntervalMinute(120) < event_time; -SET max_rows_to_read = 0; +SET max_rows_to_read = 0; -- system.text_log can be really big -- Check that we don't have too many messages formatted with fmt::runtime or strings concatenation. -- 0.001 threshold should be always enough, the value was about 0.00025 diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index bd65b937648..55f24b7e888 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -144,8 +144,7 @@ echo 9 $CLICKHOUSE_CLIENT \ --server_logs_file=/dev/null \ --query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_date >= yesterday() and message like '%find_me%'; - select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%';" --ignore-error - + select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%' SETTINGS max_rows_to_read = 0" --ignore-error echo 'finish' rm -f "$tmp_file" >/dev/null 2>&1 rm -f "$tmp_file2" >/dev/null 2>&1 diff --git a/tests/queries/0_stateless/00974_text_log_table_not_empty.sh b/tests/queries/0_stateless/00974_text_log_table_not_empty.sh index ab1b32ad90e..7fdf144a068 100755 --- a/tests/queries/0_stateless/00974_text_log_table_not_empty.sh +++ b/tests/queries/0_stateless/00974_text_log_table_not_empty.sh @@ -6,12 +6,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="SELECT 6103" -for (( i=1; i <= 50; i++ )) +for (( i=1; i <= 50; i++ )) do ${CLICKHOUSE_CLIENT} --query="SYSTEM FLUSH LOGS" sleep 0.1; -if [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() > 0 FROM system.text_log WHERE position(system.text_log.message, 'SELECT 6103') > 0 AND event_date >= yesterday()") == 1 ]]; then echo 1; exit; fi; +if [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() > 0 FROM system.text_log WHERE position(system.text_log.message, 'SELECT 6103') > 0 AND event_date >= yesterday() SETTINGS max_rows_to_read = 0") == 1 ]]; then echo 1; exit; fi; done; diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql index b8998adbc52..787d4567218 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql @@ -1,5 +1,7 @@ -- Tags: zookeeper +SET max_rows_to_read = 0; -- system.text_log can be really big + create table rmt1 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '1') order by n partition by toYYYYMMDD(d); create table rmt2 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '2') order by n partition by toYYYYMMDD(d); diff --git a/tests/queries/0_stateless/01319_query_formatting_in_server_log.sql b/tests/queries/0_stateless/01319_query_formatting_in_server_log.sql index dc88d3d48f7..4428d4fbf9c 100644 --- a/tests/queries/0_stateless/01319_query_formatting_in_server_log.sql +++ b/tests/queries/0_stateless/01319_query_formatting_in_server_log.sql @@ -2,5 +2,6 @@ SeLeCt 'ab cd' /* hello */ -- world , 1; +SET max_rows_to_read = 0; -- system.text_log can be really big SYSTEM FLUSH LOGS; SELECT extract(message, 'SeL.+?;') FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%SeLeCt \'ab\n%' ORDER BY event_time DESC LIMIT 1 FORMAT TSVRaw; diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index eca2db359bb..66cebf16662 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --max_block_siz # Now wait for cleanup thread for _ in {1..60}; do $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" - [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break; + [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%' SETTINGS max_rows_to_read = 0") -gt $((SCALE - 10)) ]] && break; sleep 1 done diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index ec318db98bf..b38d59cf6a6 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -72,6 +72,6 @@ ${CLICKHOUSE_CLIENT} --query_id "$query_id" --query "select i from simple where # We have to search the server's error log because the following warning message # is generated during pipeline destruction and thus is not sent to the client. ${CLICKHOUSE_CLIENT} --query "system flush logs" -if [[ $(${CLICKHOUSE_CLIENT} --query "select count() > 0 from system.text_log where query_id = '$query_id' and level = 'Warning' and message like '%We have query_id removed but it\'s not recorded. This is a bug%' format TSVRaw") == 1 ]]; then echo "We have query_id removed but it's not recorded. This is a bug." >&2; exit 1; fi +if [[ $(${CLICKHOUSE_CLIENT} --query "select count() > 0 from system.text_log where query_id = '$query_id' and level = 'Warning' and message like '%We have query_id removed but it\'s not recorded. This is a bug%' format TSVRaw SETTINGS max_rows_to_read = 0") == 1 ]]; then echo "We have query_id removed but it's not recorded. This is a bug." >&2; exit 1; fi ${CLICKHOUSE_CLIENT} --query "drop table simple" diff --git a/tests/queries/0_stateless/02262_column_ttl.sh b/tests/queries/0_stateless/02262_column_ttl.sh index c620d3b6d9c..30730c2a074 100755 --- a/tests/queries/0_stateless/02262_column_ttl.sh +++ b/tests/queries/0_stateless/02262_column_ttl.sh @@ -35,7 +35,7 @@ $CLICKHOUSE_CLIENT -m -q " -- OPTIMIZE TABLE x FINAL will be done in background -- attach to it's log, via table UUID in query_id (see merger/mutator code). create materialized view this_text_log engine=Memory() as - select * from system.text_log where query_id like '%${ttl_02262_uuid}%'; + select * from system.text_log where query_id like '%${ttl_02262_uuid}%' SETTINGS max_rows_to_read = 0; optimize table ttl_02262 final; system flush logs; diff --git a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql index 948ec9e9e8a..0e7a14ddf99 100644 --- a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql +++ b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql @@ -64,5 +64,6 @@ drop table rmt; drop table rmt2; system flush logs; +SET max_rows_to_read = 0; -- system.text_log can be really big select count() > 0 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts from filesystem (concurrently): Parts:%'; select count() > 1, countDistinct(thread_id) > 1 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts in blocks range%'; diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index 1548bef857f..f3f8886f691 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -99,4 +99,4 @@ $CLICKHOUSE_CLIENT -q 'system flush logs' $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', 'Query was cancelled or a client has unexpectedly dropped the connection') or - message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" + message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%') SETTINGS max_rows_to_read = 0" diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index 0597ac10cd7..c89f92ac081 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -15,6 +15,7 @@ alter table t materialize projection p_norm settings mutations_sync = 1; SYSTEM FLUSH LOGS; +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT * FROM system.text_log WHERE event_time >= now() - 30 and level == 'Error' and message like '%BAD_DATA_PART_NAME%'and message like '%p_norm%'; DROP TABLE IF EXISTS t; diff --git a/tests/queries/0_stateless/02531_two_level_aggregation_bug.sh b/tests/queries/0_stateless/02531_two_level_aggregation_bug.sh index d93fe59134f..e1bd64e19ae 100755 --- a/tests/queries/0_stateless/02531_two_level_aggregation_bug.sh +++ b/tests/queries/0_stateless/02531_two_level_aggregation_bug.sh @@ -12,5 +12,5 @@ do query_id=$(echo "select queryID() from (select sum(s), k from remote('127.0.0.{1,2}', view(select sum(number) s, bitAnd(number, 3) k from numbers_mt(1000000) group by k)) group by k) limit 1 settings group_by_two_level_threshold=1, max_threads=3, prefer_localhost_replica=1" | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @- 2>&1) ${CLICKHOUSE_CLIENT} --query="system flush logs" - ${CLICKHOUSE_CLIENT} --query="select count() from system.text_log where event_date >= yesterday() and query_id = '${query_id}' and message like '%Converting aggregation data to two-level%'" | grep -P '^6$' && break; + ${CLICKHOUSE_CLIENT} --query="select count() from system.text_log where event_date >= yesterday() and query_id = '${query_id}' and message like '%Converting aggregation data to two-level%' SETTINGS max_rows_to_read = 0" | grep -P '^6$' && break; done diff --git a/tests/queries/0_stateless/02570_fallback_from_async_insert.sh b/tests/queries/0_stateless/02570_fallback_from_async_insert.sh index d7c8944b89d..23417e5366b 100755 --- a/tests/queries/0_stateless/02570_fallback_from_async_insert.sh +++ b/tests/queries/0_stateless/02570_fallback_from_async_insert.sh @@ -48,6 +48,7 @@ $CLICKHOUSE_CLIENT --query " SELECT 'id_' || splitByChar('_', query_id)[1] AS id FROM system.text_log WHERE query_id LIKE '%$query_id_suffix' AND message LIKE '%$message%' ORDER BY id + SETTINGS max_rows_to_read = 0 " $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t_async_insert_fallback" diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql index 6b0677a80ae..8b6574562bf 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql @@ -30,6 +30,7 @@ SELECT count() FROM 02581_trips SETTINGS select_sequential_consistency = 1; DELETE FROM 02581_trips WHERE id IN (SELECT (number*10 + 9)::UInt32 FROM numbers(10000000)) SETTINGS lightweight_deletes_sync = 2; SELECT count(), _part from 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part SETTINGS select_sequential_consistency=1; +SET max_rows_to_read = 0; -- system.text_log can be really big SYSTEM FLUSH LOGS; -- Check that in every mutation there were parts that built sets (log messages like 'Created Set with 10000000 entries from 10000000 rows in 0.388989187 sec.' ) -- and parts that shared sets (log messages like 'Got set from cache in 0.388930505 sec.' ) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql index 091a9c8171d..b1facadc790 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_mutation_tasks_long.sql @@ -58,6 +58,7 @@ WHERE SETTINGS mutations_sync=2; SELECT count() from 02581_trips WHERE description = ''; +SET max_rows_to_read = 0; -- system.text_log can be really big SYSTEM FLUSH LOGS; -- Check that in every mutation there were parts that built sets (log messages like 'Created Set with 10000000 entries from 10000000 rows in 0.388989187 sec.' ) -- and parts that shared sets (log messages like 'Got set from cache in 0.388930505 sec.' ) diff --git a/tests/queries/0_stateless/02675_sparse_columns_clear_column.sql b/tests/queries/0_stateless/02675_sparse_columns_clear_column.sql index 781030ef7b4..1dac8eb23d0 100644 --- a/tests/queries/0_stateless/02675_sparse_columns_clear_column.sql +++ b/tests/queries/0_stateless/02675_sparse_columns_clear_column.sql @@ -30,5 +30,5 @@ ORDER BY column; DROP TABLE t_sparse_columns_clear SYNC; SYSTEM FLUSH LOGS; - +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT count(), groupArray(message) FROM system.text_log WHERE logger_name LIKE '%' || currentDatabase() || '.t_sparse_columns_clear' || '%' AND level = 'Error'; diff --git a/tests/queries/0_stateless/02796_calculate_text_stack_trace.sql b/tests/queries/0_stateless/02796_calculate_text_stack_trace.sql index 52d55bdbe11..dccac8fceb4 100644 --- a/tests/queries/0_stateless/02796_calculate_text_stack_trace.sql +++ b/tests/queries/0_stateless/02796_calculate_text_stack_trace.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT 'Hello', throwIf(1); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.sql b/tests/queries/0_stateless/02813_starting_in_text_log.sql index e007f58189e..a320ab7eead 100644 --- a/tests/queries/0_stateless/02813_starting_in_text_log.sql +++ b/tests/queries/0_stateless/02813_starting_in_text_log.sql @@ -1,2 +1,3 @@ SYSTEM FLUSH LOGS; +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Starting ClickHouse%'; diff --git a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql index 51301fe4ea6..5b154686d88 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql @@ -8,6 +8,7 @@ SELECT count() FROM clusterAllReplicas('test_cluster_two_shard_three_replicas_lo SYSTEM FLUSH LOGS; SET enable_parallel_replicas=0; +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT count() > 0 FROM system.text_log WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_190aed82-2423-413b-ad4c-24dcca50f65b') AND message LIKE '%Parallel reading from replicas is disabled for cluster%'; diff --git a/tests/queries/0_stateless/02875_parallel_replicas_remote.sql b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql index 259ed02b2a3..08afeb97708 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_remote.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql @@ -8,6 +8,7 @@ SELECT count() FROM remote('127.0.0.{1..6}', currentDatabase(), tt) settings log SYSTEM FLUSH LOGS; SET enable_parallel_replicas=0; +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT count() > 0 FROM system.text_log WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_89f3c39b-1919-48cb-b66e-ef9904e73146') AND message LIKE '%Parallel reading from replicas is disabled for cluster%'; diff --git a/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql index dc500aaff3b..a745625f17a 100644 --- a/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql +++ b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql @@ -34,6 +34,7 @@ SELECT count() FROM checksums_r3; SYSTEM FLUSH LOGS; +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT * FROM system.text_log WHERE event_time >= now() - INTERVAL 120 SECOND and level == 'Error' and message like '%CHECKSUM_DOESNT_MATCH%' and logger_name like ('%' || currentDatabase() || '%checksums_r%'); DROP TABLE IF EXISTS checksums_r3; diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql index cbb4ff5334c..07c5a8329be 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql @@ -21,6 +21,7 @@ SELECT count(), min(k), max(k), avg(k) FROM t1 SETTINGS log_comment='02898_defau -- check logs SYSTEM FLUSH LOGS; +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT count() > 0 FROM system.text_log WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_default_190aed82-2423-413b-ad4c-24dcca50f65b') AND message LIKE '%Total rows to read: 3000%' SETTINGS enable_parallel_replicas=0; diff --git a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh index b77e5b0b402..aaecc7301e4 100755 --- a/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh +++ b/tests/queries/0_stateless/02903_rmt_retriable_merge_exception.sh @@ -56,7 +56,8 @@ for _ in {0..50}; do ( (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{$table_uuid::$part_name}%No active replica has part $part_name or covering part%') or (logger_name like '$table_uuid::$part_name (MergeFromLogEntryTask)' and message like '%No active replica has part $part_name or covering part%') - ); + ) + SETTINGS max_rows_to_read = 0; ") if [[ $no_active_repilica_messages -gt 0 ]]; then break @@ -78,5 +79,6 @@ $CLICKHOUSE_CLIENT -m -q " (logger_name = 'MergeTreeBackgroundExecutor' and message like '%{$table_uuid::$part_name}%No active replica has part $part_name or covering part%') or (logger_name like '$table_uuid::$part_name (MergeFromLogEntryTask)' and message like '%No active replica has part $part_name or covering part%') ) - group by level; + group by level + SETTINGS max_rows_to_read = 0; " diff --git a/tests/queries/0_stateless/02935_parallel_replicas_settings.sql b/tests/queries/0_stateless/02935_parallel_replicas_settings.sql index 5e3b1a13f1a..ccf1caddbb9 100644 --- a/tests/queries/0_stateless/02935_parallel_replicas_settings.sql +++ b/tests/queries/0_stateless/02935_parallel_replicas_settings.sql @@ -11,7 +11,7 @@ SET cluster_for_parallel_replicas='parallel_replicas'; SELECT count() FROM test_parallel_replicas_settings WHERE NOT ignore(*) settings log_comment='0_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f'; SYSTEM FLUSH LOGS; - +SET max_rows_to_read = 0; -- system.text_log can be really big SELECT count() > 0 FROM system.text_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where current_database=currentDatabase() AND log_comment='0_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f') diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql index bb036a6c133..a08f35cfc1d 100644 --- a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -6,6 +6,7 @@ select conut(); -- { serverError UNKNOWN_FUNCTION } system flush logs; +SET max_rows_to_read = 0; -- system.text_log can be really big select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier {} in scope {}{}' and value1 = 'expression' and value3 = '`count`' and value4 = 'SELECT count'; diff --git a/tests/queries/0_stateless/03141_fetches_errors_stress.sql b/tests/queries/0_stateless/03141_fetches_errors_stress.sql index 2f6b0b08574..69829a44b58 100644 --- a/tests/queries/0_stateless/03141_fetches_errors_stress.sql +++ b/tests/queries/0_stateless/03141_fetches_errors_stress.sql @@ -11,6 +11,7 @@ system disable failpoint replicated_sends_failpoint; system sync replica data_r2; system flush logs; +SET max_rows_to_read = 0; -- system.text_log can be really big select event_time_microseconds, logger_name, message from system.text_log where level = 'Error' and message like '%Malformed chunked encoding%' order by 1 format LineAsString; -- { echoOn } diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql index 96221f27e73..968cc4a40a7 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_1.sql @@ -24,7 +24,7 @@ SELECT count() FROM t_ind_merge_1 WHERE b < 100 SETTINGS force_data_skipping_ind EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge_1 WHERE b < 100; SYSTEM FLUSH LOGS; - +SET max_rows_to_read = 0; -- system.text_log can be really big WITH (SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge_1') AS uuid, extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups diff --git a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql index b749e0c84b0..bf145d875fd 100644 --- a/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql +++ b/tests/queries/0_stateless/03166_skip_indexes_vertical_merge_2.sql @@ -26,6 +26,7 @@ INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand() OPTIMIZE TABLE t_ind_merge_2 FINAL; SYSTEM FLUSH LOGS; +SET max_rows_to_read = 0; -- system.text_log can be really big --- merged: a, c, d; gathered: b, e, f WITH diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index 6e155aee1df..a2782083e7b 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -15,7 +15,8 @@ check_replicas_read_in_order() { SELECT COUNT() > 0 FROM system.text_log WHERE query_id IN (SELECT query_id FROM system.query_log WHERE query_id != '$1' AND initial_query_id = '$1' AND event_date >= yesterday()) - AND event_date >= yesterday() AND message ILIKE '%Reading%ranges in order%'" + AND event_date >= yesterday() AND message ILIKE '%Reading%ranges in order%' + SETTINGS max_rows_to_read=0" } # replicas should use reading in order following initiator's decision to execute aggregation in order. From ddf3259d27df236824dfb800b8d18d46e425e7ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 14:02:15 +0200 Subject: [PATCH 31/53] Fix style --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fdb5dd82561..1ba88025277 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -292,7 +292,7 @@ def clickhouse_execute_http( "http_receive_timeout": timeout, "http_send_timeout": timeout, "output_format_parallel_formatting": 0, - "max_rows_to_read": 0 # Some queries read from system.text_log which might get too big + "max_rows_to_read": 0, # Some queries read from system.text_log which might get too big } if settings is not None: params.update(settings) From b3acdb4f1c80b7cd58d323d50bd1240e17748000 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 14:19:40 +0200 Subject: [PATCH 32/53] Fix creation of stateful window functions on misaligned memory --- src/AggregateFunctions/WindowFunction.h | 23 +++++++--- src/Processors/Transforms/WindowTransform.cpp | 42 ++++++++----------- ..._unaligned_window_function_state.reference | 0 .../03251_unaligned_window_function_state.sql | 3 ++ 4 files changed, 38 insertions(+), 30 deletions(-) create mode 100644 tests/queries/0_stateless/03251_unaligned_window_function_state.reference create mode 100644 tests/queries/0_stateless/03251_unaligned_window_function_state.sql diff --git a/src/AggregateFunctions/WindowFunction.h b/src/AggregateFunctions/WindowFunction.h index f7fbd7389ea..be214f4c824 100644 --- a/src/AggregateFunctions/WindowFunction.h +++ b/src/AggregateFunctions/WindowFunction.h @@ -78,11 +78,6 @@ struct WindowFunction : public IAggregateFunctionHelper, public } String getName() const override { return name; } - void create(AggregateDataPtr __restrict) const override { } - void destroy(AggregateDataPtr __restrict) const noexcept override { } - bool hasTrivialDestructor() const override { return true; } - size_t sizeOfData() const override { return 0; } - size_t alignOfData() const override { return 1; } void add(AggregateDataPtr __restrict, const IColumn **, size_t, Arena *) const override { fail(); } void merge(AggregateDataPtr __restrict, ConstAggregateDataPtr, Arena *) const override { fail(); } void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override { fail(); } @@ -90,6 +85,22 @@ struct WindowFunction : public IAggregateFunctionHelper, public void insertResultInto(AggregateDataPtr __restrict, IColumn &, Arena *) const override { fail(); } }; +struct StatelessWindowFunction : public WindowFunction +{ + StatelessWindowFunction( + const std::string & name_, const DataTypes & argument_types_, const Array & parameters_, const DataTypePtr & result_type_) + : WindowFunction(name_, argument_types_, parameters_, result_type_) + { + } + + size_t sizeOfData() const override { return 0; } + size_t alignOfData() const override { return 1; } + + void create(AggregateDataPtr __restrict) const override { } + void destroy(AggregateDataPtr __restrict) const noexcept override { } + bool hasTrivialDestructor() const override { return true; } +}; + template struct StatefulWindowFunction : public WindowFunction { @@ -100,7 +111,7 @@ struct StatefulWindowFunction : public WindowFunction } size_t sizeOfData() const override { return sizeof(State); } - size_t alignOfData() const override { return 1; } + size_t alignOfData() const override { return alignof(State); } void create(AggregateDataPtr __restrict place) const override { new (place) State(); } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 6f43a8e4b24..37948cd55e7 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1495,11 +1495,10 @@ void WindowTransform::work() } } -struct WindowFunctionRank final : public WindowFunction +struct WindowFunctionRank final : public StatelessWindowFunction { - WindowFunctionRank(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) - : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) + WindowFunctionRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + : StatelessWindowFunction(name_, argument_types_, parameters_, std::make_shared()) {} bool allocatesMemoryInArena() const override { return false; } @@ -1514,11 +1513,10 @@ struct WindowFunctionRank final : public WindowFunction } }; -struct WindowFunctionDenseRank final : public WindowFunction +struct WindowFunctionDenseRank final : public StatelessWindowFunction { - WindowFunctionDenseRank(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) - : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) + WindowFunctionDenseRank(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + : StatelessWindowFunction(name_, argument_types_, parameters_, std::make_shared()) {} bool allocatesMemoryInArena() const override { return false; } @@ -1716,7 +1714,7 @@ struct WindowFunctionExponentialTimeDecayedSum final : public StatefulWindowFunc const Float64 decay_length; }; -struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction +struct WindowFunctionExponentialTimeDecayedMax final : public StatelessWindowFunction { static constexpr size_t ARGUMENT_VALUE = 0; static constexpr size_t ARGUMENT_TIME = 1; @@ -1731,9 +1729,8 @@ struct WindowFunctionExponentialTimeDecayedMax final : public WindowFunction return applyVisitor(FieldVisitorConvertToNumber(), parameters_[0]); } - WindowFunctionExponentialTimeDecayedMax(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) - : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) + WindowFunctionExponentialTimeDecayedMax(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + : StatelessWindowFunction(name_, argument_types_, parameters_, std::make_shared()) , decay_length(getDecayLength(parameters_, name_)) { if (argument_types.size() != 2) @@ -1991,11 +1988,10 @@ struct WindowFunctionExponentialTimeDecayedAvg final : public StatefulWindowFunc const Float64 decay_length; }; -struct WindowFunctionRowNumber final : public WindowFunction +struct WindowFunctionRowNumber final : public StatelessWindowFunction { - WindowFunctionRowNumber(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) - : WindowFunction(name_, argument_types_, parameters_, std::make_shared()) + WindowFunctionRowNumber(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + : StatelessWindowFunction(name_, argument_types_, parameters_, std::make_shared()) {} bool allocatesMemoryInArena() const override { return false; } @@ -2273,13 +2269,12 @@ public: // ClickHouse-specific variant of lag/lead that respects the window frame. template -struct WindowFunctionLagLeadInFrame final : public WindowFunction +struct WindowFunctionLagLeadInFrame final : public StatelessWindowFunction { FunctionBasePtr func_cast = nullptr; - WindowFunctionLagLeadInFrame(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) - : WindowFunction(name_, argument_types_, parameters_, createResultType(argument_types_, name_)) + WindowFunctionLagLeadInFrame(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + : StatelessWindowFunction(name_, argument_types_, parameters_, createResultType(argument_types_, name_)) { if (!parameters.empty()) { @@ -2427,11 +2422,10 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction } }; -struct WindowFunctionNthValue final : public WindowFunction +struct WindowFunctionNthValue final : public StatelessWindowFunction { - WindowFunctionNthValue(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) - : WindowFunction(name_, argument_types_, parameters_, createResultType(name_, argument_types_)) + WindowFunctionNthValue(const std::string & name_, const DataTypes & argument_types_, const Array & parameters_) + : StatelessWindowFunction(name_, argument_types_, parameters_, createResultType(name_, argument_types_)) { if (!parameters.empty()) { diff --git a/tests/queries/0_stateless/03251_unaligned_window_function_state.reference b/tests/queries/0_stateless/03251_unaligned_window_function_state.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03251_unaligned_window_function_state.sql b/tests/queries/0_stateless/03251_unaligned_window_function_state.sql new file mode 100644 index 00000000000..2ff75f61f76 --- /dev/null +++ b/tests/queries/0_stateless/03251_unaligned_window_function_state.sql @@ -0,0 +1,3 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/70569 +-- Reproduces UBSAN alert about misaligned address +SELECT anyLast(id), anyLast(time), exponentialTimeDecayedAvg(10)(id, time) FROM values('id Int8, time DateTime', (1,1),(1,2),(2,3),(3,3),(3,5)); -- { serverError BAD_ARGUMENTS } From 160cdbb879e11c6d4bc499a0e8ebf3c7bb951c98 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 13:30:19 +0000 Subject: [PATCH 33/53] Uniformize ErrorCodes.cpp --- src/Common/ErrorCodes.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 9f07c3ed5d5..3f4a75fae3c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -614,6 +614,7 @@ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ + M(902, PROTOCOL_VERSION_MISMATCH) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From 30d148ea811ab54c91586beee775738ff2bdbd59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 13:39:47 +0000 Subject: [PATCH 34/53] Small diff --- src/Client/ConnectionParameters.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index 382bfe34a3d..85e5fcb0ce1 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -15,7 +15,7 @@ namespace DB { struct ConnectionParameters { - std::string host; + String host; UInt16 port{}; std::string default_database; std::string user; @@ -30,8 +30,8 @@ struct ConnectionParameters ConnectionTimeouts timeouts; ConnectionParameters() = default; - ConnectionParameters(const Poco::Util::AbstractConfiguration & config, std::string host); - ConnectionParameters(const Poco::Util::AbstractConfiguration & config, std::string host, std::optional port); + ConnectionParameters(const Poco::Util::AbstractConfiguration & config, String host); + ConnectionParameters(const Poco::Util::AbstractConfiguration & config, String host, std::optional port); static UInt16 getPortFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & connection_host); From 8215d6bea767d5580226caefa46d1902436cb326 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 13:40:19 +0000 Subject: [PATCH 35/53] Unify CurrentMetrics.cpp --- src/Common/CurrentMetrics.cpp | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 658eaedbda1..64877196aef 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -291,9 +291,14 @@ M(CacheWarmerBytesInProgress, "Total size of remote file segments waiting to be asynchronously loaded into filesystem cache.") \ M(DistrCacheOpenedConnections, "Number of open connections to Distributed Cache") \ M(DistrCacheUsedConnections, "Number of currently used connections to Distributed Cache") \ + M(DistrCacheAllocatedConnections, "Number of currently allocated connections to Distributed Cache connection pool") \ + M(DistrCacheBorrowedConnections, "Number of currently borrowed connections to Distributed Cache connection pool") \ M(DistrCacheReadRequests, "Number of executed Read requests to Distributed Cache") \ M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \ M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") \ + M(DistrCacheRegisteredServers, "Number of distributed cache registered servers") \ + M(DistrCacheRegisteredServersCurrentAZ, "Number of distributed cache registered servers in current az") \ + M(DistrCacheServerS3CachedClients, "Number of distributed cache S3 cached clients") \ \ M(SchedulerIOReadScheduled, "Number of IO reads are being scheduled currently") \ M(SchedulerIOWriteScheduled, "Number of IO writes are being scheduled currently") \ @@ -314,6 +319,20 @@ M(FilteringMarksWithSecondaryKeys, "Number of threads currently doing filtering of mark ranges by secondary keys") \ \ M(DiskS3NoSuchKeyErrors, "The number of `NoSuchKey` errors that occur when reading data from S3 cloud storage through ClickHouse disks.") \ + \ + M(SharedCatalogStateApplicationThreads, "Number of threads in the threadpool for state application in Shared Catalog.") \ + M(SharedCatalogStateApplicationThreadsActive, "Number of active threads in the threadpool for state application in Shared Catalog.") \ + M(SharedCatalogStateApplicationThreadsScheduled, "Number of queued or active jobs in the threadpool for state application in Shared Catalog.") \ + \ + M(SharedCatalogDropLocalThreads, "Number of threads in the threadpool for drop of local tables in Shared Catalog.") \ + M(SharedCatalogDropLocalThreadsActive, "Number of active threads in the threadpool for drop of local tables in Shared Catalog.") \ + M(SharedCatalogDropLocalThreadsScheduled, "Number of queued or active jobs in the threadpool for drop of local tables in Shared Catalog.") \ + \ + M(SharedCatalogDropZooKeeperThreads, "Number of threads in the threadpool for drop of object in ZooKeeper in Shared Catalog.") \ + M(SharedCatalogDropZooKeeperThreadsActive, "Number of active threads in the threadpool for drop of object in ZooKeeper in Shared Catalog.") \ + M(SharedCatalogDropZooKeeperThreadsScheduled, "Number of queued or active jobs in the threadpool for drop of object in ZooKeeper in Shared Catalog.") \ + \ + M(SharedDatabaseCatalogTablesInLocalDropDetachQueue, "Number of tables in the queue for local drop or detach in Shared Catalog.") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) From 623a620705b1695499658ca40eddde63cb8561fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 13:41:20 +0000 Subject: [PATCH 36/53] Unify failpoints --- src/Common/FailPoint.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 5978dbc18d2..0898bdded83 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -49,11 +49,21 @@ static struct InitFiu ONCE(smt_commit_write_zk_fail_before_op) \ ONCE(smt_commit_merge_change_version_before_op) \ ONCE(smt_merge_mutate_intention_freeze_in_destructor) \ + ONCE(smt_add_part_sleep_after_add_before_commit) \ + ONCE(smt_sleep_in_constructor) \ ONCE(meta_in_keeper_create_metadata_failure) \ + ONCE(smt_insert_retry_timeout) \ + ONCE(smt_insert_fake_hardware_error) \ + ONCE(smt_sleep_after_hardware_in_insert) \ + ONCE(smt_throw_keeper_exception_after_successful_insert) \ + REGULAR(smt_dont_merge_first_part) \ + REGULAR(smt_sleep_in_schedule_data_processing_job) \ REGULAR(cache_warmer_stall) \ REGULAR(check_table_query_delay_for_part) \ REGULAR(dummy_failpoint) \ REGULAR(prefetched_reader_pool_failpoint) \ + REGULAR(shared_set_sleep_during_update) \ + REGULAR(smt_outdated_parts_exception_response) \ PAUSEABLE_ONCE(replicated_merge_tree_insert_retry_pause) \ PAUSEABLE_ONCE(finish_set_quorum_failed_parts) \ PAUSEABLE_ONCE(finish_clean_quorum_failed_parts) \ From 984dfd4fab376f7e17c90a8b9f07b66a548185ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 13:43:28 +0000 Subject: [PATCH 37/53] Unify ProfileEvents --- src/Common/ProfileEvents.cpp | 98 +++++++++++++++++++++++++++--------- src/Common/ProfileEvents.h | 3 ++ 2 files changed, 78 insertions(+), 23 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index eff8206e676..29c538710ac 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -241,6 +241,8 @@ M(MergeVerticalStageExecuteMilliseconds, "Total busy time spent for execution of vertical stage of background merges", ValueType::Milliseconds) \ M(MergeProjectionStageTotalMilliseconds, "Total time spent for projection stage of background merges", ValueType::Milliseconds) \ M(MergeProjectionStageExecuteMilliseconds, "Total busy time spent for execution of projection stage of background merges", ValueType::Milliseconds) \ + M(MergePrewarmStageTotalMilliseconds, "Total time spent for prewarm stage of background merges", ValueType::Milliseconds) \ + M(MergePrewarmStageExecuteMilliseconds, "Total busy time spent for execution of prewarm stage of background merges", ValueType::Milliseconds) \ \ M(MergingSortedMilliseconds, "Total time spent while merging sorted columns", ValueType::Milliseconds) \ M(AggregatingSortedMilliseconds, "Total time spent while aggregating sorted columns", ValueType::Milliseconds) \ @@ -639,6 +641,8 @@ The server successfully detected this situation and will download merged part fr M(MetadataFromKeeperBackgroundCleanupTransactions, "Number of times old transaction idempotency token was cleaned up by background task", ValueType::Number) \ M(MetadataFromKeeperBackgroundCleanupErrors, "Number of times an error was encountered in background cleanup task", ValueType::Number) \ \ + M(SharedMergeTreeMetadataCacheHintLoadedFromCache, "Number of times metadata cache hint was found without going to Keeper", ValueType::Number) \ + \ M(KafkaRebalanceRevocations, "Number of partition revocations (the first stage of consumer group rebalance)", ValueType::Number) \ M(KafkaRebalanceAssignments, "Number of partition assignments (the final stage of consumer group rebalance)", ValueType::Number) \ M(KafkaRebalanceErrors, "Number of failed consumer group rebalances", ValueType::Number) \ @@ -742,29 +746,51 @@ The server successfully detected this situation and will download merged part fr M(ConnectionPoolIsFullMicroseconds, "Total time spent waiting for a slot in connection pool.", ValueType::Microseconds) \ M(AsyncLoaderWaitMicroseconds, "Total time a query was waiting for async loader jobs.", ValueType::Microseconds) \ \ - M(DistrCacheServerSwitches, "Number of server switches between distributed cache servers in read/write-through cache", ValueType::Number) \ - M(DistrCacheReadMicroseconds, "Time spent reading from distributed cache", ValueType::Microseconds) \ - M(DistrCacheFallbackReadMicroseconds, "Time spend reading from fallback buffer instead of distribted cache", ValueType::Microseconds) \ - M(DistrCachePrecomputeRangesMicroseconds, "Time spent to precompute read ranges", ValueType::Microseconds) \ - M(DistrCacheNextImplMicroseconds, "Time spend in ReadBufferFromDistributedCache::nextImpl", ValueType::Microseconds) \ - M(DistrCacheOpenedConnections, "The number of open connections to distributed cache", ValueType::Number) \ - M(DistrCacheReusedConnections, "The number of reused connections to distributed cache", ValueType::Number) \ - M(DistrCacheHoldConnections, "The number of used connections to distributed cache", ValueType::Number) \ + M(DistrCacheServerSwitches, "Distributed Cache read buffer event. Number of server switches between distributed cache servers in read/write-through cache", ValueType::Number) \ + M(DistrCacheReadMicroseconds, "Distributed Cache read buffer event. Time spent reading from distributed cache", ValueType::Microseconds) \ + M(DistrCacheFallbackReadMicroseconds, "Distributed Cache read buffer event. Time spend reading from fallback buffer instead of distributed cache", ValueType::Microseconds) \ + M(DistrCachePrecomputeRangesMicroseconds, "Distributed Cache read buffer event. Time spent to precompute read ranges", ValueType::Microseconds) \ + M(DistrCacheNextImplMicroseconds, "Distributed Cache read buffer event. Time spend in ReadBufferFromDistributedCache::nextImpl", ValueType::Microseconds) \ + M(DistrCacheStartRangeMicroseconds, "Distributed Cache read buffer event. Time spent to start a new read range with distributed cache", ValueType::Microseconds) \ + M(DistrCacheIgnoredBytesWhileWaitingProfileEvents, "Distributed Cache read buffer event. Ignored bytes while waiting for profile events in distributed cache", ValueType::Number) \ + M(DistrCacheRangeChange, "Distributed Cache read buffer event. Number of times we changed read range because of seek/last_position change", ValueType::Number) \ \ - M(DistrCacheGetResponseMicroseconds, "Time spend to wait for response from distributed cache", ValueType::Microseconds) \ - M(DistrCacheStartRangeMicroseconds, "Time spent to start a new read range with distributed cache", ValueType::Microseconds) \ - M(DistrCacheLockRegistryMicroseconds, "Time spent to take DistributedCacheRegistry lock", ValueType::Microseconds) \ - M(DistrCacheUnusedPackets, "Number of skipped unused packets from distributed cache", ValueType::Number) \ - M(DistrCachePackets, "Total number of packets received from distributed cache", ValueType::Number) \ - M(DistrCacheUnusedPacketsBytes, "The number of bytes in Data packets which were ignored", ValueType::Bytes) \ - M(DistrCacheRegistryUpdateMicroseconds, "Time spent updating distributed cache registry", ValueType::Microseconds) \ - M(DistrCacheRegistryUpdates, "Number of distributed cache registry updates", ValueType::Number) \ + M(DistrCacheGetResponseMicroseconds, "Distributed Cache client event. Time spend to wait for response from distributed cache", ValueType::Microseconds) \ + M(DistrCacheReadErrors, "Distributed Cache client event. Number of distributed cache errors during read", ValueType::Number) \ + M(DistrCacheMakeRequestErrors, "Distributed Cache client event. Number of distributed cache errors when making a request", ValueType::Number) \ + M(DistrCacheReceiveResponseErrors, "Distributed Cache client event. Number of distributed cache errors when receiving response a request", ValueType::Number) \ \ - M(DistrCacheConnectMicroseconds, "The time spent to connect to distributed cache", ValueType::Microseconds) \ - M(DistrCacheConnectAttempts, "The number of connection attempts to distributed cache", ValueType::Number) \ - M(DistrCacheGetClient, "Number of client access times", ValueType::Number) \ + M(DistrCachePackets, "Distributed Cache client event. Total number of packets received from distributed cache", ValueType::Number) \ + M(DistrCachePacketsBytes, "Distributed Cache client event. The number of bytes in Data packets which were not ignored", ValueType::Bytes) \ + M(DistrCacheUnusedPackets, "Distributed Cache client event. Number of skipped unused packets from distributed cache", ValueType::Number) \ + M(DistrCacheUnusedPacketsBytes, "Distributed Cache client event. The number of bytes in Data packets which were ignored", ValueType::Bytes) \ + M(DistrCacheUnusedPacketsBufferAllocations, "Distributed Cache client event. The number of extra buffer allocations in case we could not reuse existing buffer", ValueType::Number) \ \ - M(DistrCacheServerProcessRequestMicroseconds, "Time spent processing request on DistributedCache server side", ValueType::Microseconds) \ + M(DistrCacheLockRegistryMicroseconds, "Distributed Cache registry event. Time spent to take DistributedCacheRegistry lock", ValueType::Microseconds) \ + M(DistrCacheRegistryUpdateMicroseconds, "Distributed Cache registry event. Time spent updating distributed cache registry", ValueType::Microseconds) \ + M(DistrCacheRegistryUpdates, "Distributed Cache registry event. Number of distributed cache registry updates", ValueType::Number) \ + M(DistrCacheHashRingRebuilds, "Distributed Cache registry event. Number of distributed cache hash ring rebuilds", ValueType::Number) \ + \ + M(DistrCacheReadBytesFromCache, "Distributed Cache read buffer event. Bytes read from distributed cache", ValueType::Bytes) \ + M(DistrCacheReadBytesFromFallbackBuffer, "Distributed Cache read buffer event. Bytes read from fallback buffer", ValueType::Number) \ + \ + M(DistrCacheRangeResetBackward, "Distributed Cache read buffer event. Number of times we reset read range because of seek/last_position change", ValueType::Number) \ + M(DistrCacheRangeResetForward, "Distributed Cache read buffer event. Number of times we reset read range because of seek/last_position change", ValueType::Number) \ + \ + M(DistrCacheOpenedConnections, "Distributed Cache connection event. The number of open connections to distributed cache", ValueType::Number) \ + M(DistrCacheReusedConnections, "Distributed Cache connection event. The number of reused connections to distributed cache", ValueType::Number) \ + M(DistrCacheOpenedConnectionsBypassingPool, "Distributed Cache connection event. The number of open connections to distributed cache bypassing pool", ValueType::Number) \ + M(DistrCacheConnectMicroseconds, "Distributed Cache connection event. The time spent to connect to distributed cache", ValueType::Microseconds) \ + M(DistrCacheConnectAttempts, "Distributed Cache connection event. The number of connection attempts to distributed cache", ValueType::Number) \ + M(DistrCacheGetClientMicroseconds, "Distributed Cache connection event. Time spent getting client for distributed cache", ValueType::Microseconds) \ + \ + M(DistrCacheServerProcessRequestMicroseconds, "Distributed Cache server event. Time spent processing request on DistributedCache server side", ValueType::Microseconds) \ + M(DistrCacheServerStartRequestPackets, "Distributed Cache server event. Number of StartRequest packets in DistributedCacheServer", ValueType::Number) \ + M(DistrCacheServerContinueRequestPackets, "Distributed Cache server event. Number of ContinueRequest packets in DistributedCacheServer", ValueType::Number) \ + M(DistrCacheServerEndRequestPackets, "Distributed Cache server event. Number of EndRequest packets in DistributedCacheServer", ValueType::Number) \ + M(DistrCacheServerAckRequestPackets, "Distributed Cache server event. Number of AckRequest packets in DistributedCacheServer", ValueType::Number) \ + M(DistrCacheServerNewS3CachedClients, "Distributed Cache server event. The number of new cached s3 clients", ValueType::Number) \ + M(DistrCacheServerReusedS3CachedClients, "Distributed Cache server event. The number of reused cached s3 clients", ValueType::Number) \ \ M(LogTest, "Number of log messages with level Test", ValueType::Number) \ M(LogTrace, "Number of log messages with level Trace", ValueType::Number) \ @@ -788,15 +814,38 @@ The server successfully detected this situation and will download merged part fr M(InterfacePostgreSQLReceiveBytes, "Number of bytes received through PostgreSQL interfaces", ValueType::Bytes) \ \ M(ParallelReplicasUsedCount, "Number of replicas used to execute a query with task-based parallel replicas", ValueType::Number) \ + M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas", ValueType::Number) \ + M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but found to be unavailable during query execution with task-based parallel replicas", ValueType::Number) \ \ + M(SharedMergeTreeVirtualPartsUpdates, "Virtual parts update count", ValueType::Number) \ + M(SharedMergeTreeVirtualPartsUpdatesByLeader, "Virtual parts updates by leader", ValueType::Number) \ + M(SharedMergeTreeVirtualPartsUpdateMicroseconds, "Virtual parts update microseconds", ValueType::Microseconds) \ + M(SharedMergeTreeVirtualPartsUpdatesFromZooKeeper, "Virtual parts updates count from ZooKeeper", ValueType::Number) \ + M(SharedMergeTreeVirtualPartsUpdatesFromZooKeeperMicroseconds, "Virtual parts updates from ZooKeeper microseconds", ValueType::Microseconds) \ + M(SharedMergeTreeVirtualPartsUpdatesPeerNotFound, "Virtual updates from peer failed because no one found", ValueType::Number) \ + M(SharedMergeTreeVirtualPartsUpdatesFromPeer, "Virtual parts updates count from peer", ValueType::Number) \ + M(SharedMergeTreeVirtualPartsUpdatesFromPeerMicroseconds, "Virtual parts updates from peer microseconds", ValueType::Microseconds) \ + M(SharedMergeTreeVirtualPartsUpdatesForMergesOrStatus, "Virtual parts updates from non-default background job", ValueType::Number) \ + M(SharedMergeTreeVirtualPartsUpdatesLeaderFailedElection, "Virtual parts updates leader election failed", ValueType::Number) \ + M(SharedMergeTreeVirtualPartsUpdatesLeaderSuccessfulElection, "Virtual parts updates leader election successful", ValueType::Number) \ + M(SharedMergeTreeMergeMutationAssignmentAttempt, "How many times we tried to assign merge or mutation", ValueType::Number) \ + M(SharedMergeTreeMergeMutationAssignmentFailedWithNothingToDo, "How many times we tried to assign merge or mutation and failed because nothing to merge", ValueType::Number) \ + M(SharedMergeTreeMergeMutationAssignmentFailedWithConflict, "How many times we tried to assign merge or mutation and failed because of conflict in Keeper", ValueType::Number) \ + M(SharedMergeTreeMergeMutationAssignmentSuccessful, "How many times we tried to assign merge or mutation", ValueType::Number) \ + M(SharedMergeTreeMergePartsMovedToOudated, "How many parts moved to oudated directory", ValueType::Number) \ + M(SharedMergeTreeMergePartsMovedToCondemned, "How many parts moved to condemned directory", ValueType::Number) \ + M(SharedMergeTreeOutdatedPartsConfirmationRequest, "How many ZooKeeper requests were used to config outdated parts", ValueType::Number) \ + M(SharedMergeTreeOutdatedPartsConfirmationInvocations, "How many invocations were made to confirm outdated parts", ValueType::Number) \ + M(SharedMergeTreeOutdatedPartsHTTPRequest, "How many HTTP requests were send to confirm outdated parts", ValueType::Number) \ + M(SharedMergeTreeOutdatedPartsHTTPResponse, "How many HTTP responses were send to confirm outdated parts", ValueType::Number) \ + M(SharedMergeTreeCondemnedPartsKillRequest, "How many ZooKeeper requests were used to remove condemned parts", ValueType::Number) \ + M(SharedMergeTreeCondemnedPartsLockConfict, "How many times we failed to acquite lock because of conflict", ValueType::Number) \ + M(SharedMergeTreeCondemnedPartsRemoved, "How many condemned parts were removed", ValueType::Number) \ M(KeeperLogsEntryReadFromLatestCache, "Number of log entries in Keeper being read from latest logs cache", ValueType::Number) \ M(KeeperLogsEntryReadFromCommitCache, "Number of log entries in Keeper being read from commit logs cache", ValueType::Number) \ M(KeeperLogsEntryReadFromFile, "Number of log entries in Keeper being read directly from the changelog file", ValueType::Number) \ M(KeeperLogsPrefetchedEntries, "Number of log entries in Keeper being prefetched from the changelog file", ValueType::Number) \ \ - M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas", ValueType::Number) \ - M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but found to be unavailable during query execution with task-based parallel replicas", ValueType::Number) \ - \ M(StorageConnectionsCreated, "Number of created connections for storages", ValueType::Number) \ M(StorageConnectionsReused, "Number of reused connections for storages", ValueType::Number) \ M(StorageConnectionsReset, "Number of reset connections for storages", ValueType::Number) \ @@ -828,6 +877,9 @@ The server successfully detected this situation and will download merged part fr M(ReadWriteBufferFromHTTPRequestsSent, "Number of HTTP requests sent by ReadWriteBufferFromHTTP", ValueType::Number) \ M(ReadWriteBufferFromHTTPBytes, "Total size of payload bytes received and sent by ReadWriteBufferFromHTTP. Doesn't include HTTP headers.", ValueType::Bytes) \ \ + M(SharedDatabaseCatalogFailedToApplyState, "Number of failures to apply new state in SharedDatabaseCatalog", ValueType::Number) \ + M(SharedDatabaseCatalogStateApplicationMicroseconds, "Total time spend on application of new state in SharedDatabaseCatalog", ValueType::Microseconds) \ + \ M(GWPAsanAllocateSuccess, "Number of successful allocations done by GWPAsan", ValueType::Number) \ M(GWPAsanAllocateFailed, "Number of failed allocations done by GWPAsan (i.e. filled pool)", ValueType::Number) \ M(GWPAsanFree, "Number of free operations done by GWPAsan", ValueType::Number) \ diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 50d6a5e1a18..94551c23ac0 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -38,6 +38,9 @@ namespace ProfileEvents }; Timer(Counters & counters_, Event timer_event_, Resolution resolution_); Timer(Counters & counters_, Event timer_event_, Event counter_event, Resolution resolution_); + Timer(Timer && other) noexcept + : counters(other.counters), timer_event(std::move(other.timer_event)), watch(std::move(other.watch)), resolution(std::move(other.resolution)) + {} ~Timer() { end(); } void cancel() { watch.reset(); } void restart() { watch.restart(); } From 6f635a18f7677a8d63ce06d49582e4b52853934b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 14:07:09 +0000 Subject: [PATCH 38/53] Unify more code --- src/Common/ZooKeeper/ZooKeeperRetries.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperRetries.h b/src/Common/ZooKeeper/ZooKeeperRetries.h index 39782d08b33..b5b03971385 100644 --- a/src/Common/ZooKeeper/ZooKeeperRetries.h +++ b/src/Common/ZooKeeper/ZooKeeperRetries.h @@ -159,6 +159,8 @@ public: const std::string & getLastKeeperErrorMessage() const { return keeper_error.message; } /// action will be called only once and only after latest failed retry + /// NOTE: this one will be called only in case when retries finishes with Keeper exception + /// if it will be some other exception this function will not be called. void actionAfterLastFailedRetry(std::function f) { action_after_last_failed_retry = std::move(f); } const std::string & getName() const { return name; } From ac492d7db00d13d24f4bd55178548db01a1365ed Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 14 Oct 2024 16:37:35 +0200 Subject: [PATCH 39/53] Update 03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh --- .../03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh b/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh index d027a77907c..aae213a3d53 100755 --- a/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh +++ b/tests/queries/0_stateless/03250_SYSTEM_DROP_FORMAT_SCHEMA_CACHE_FOR_Protobuf.sh @@ -13,8 +13,8 @@ cp -r $CLIENT_SCHEMADIR/03250.proto $SCHEMADIR/$SERVER_SCHEMADIR/ $CLICKHOUSE_CLIENT --query "SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf" -export BINARY_FILE_PATH BINARY_FILE_PATH=$(mktemp "$CLICKHOUSE_USER_FILES/03250.XXXXXX.binary") +export BINARY_FILE_PATH $CLICKHOUSE_CLIENT --query "SELECT * FROM numbers(10) FORMAT Protobuf SETTINGS format_schema = '$CLIENT_SCHEMADIR/03250:Numbers'" > $BINARY_FILE_PATH chmod 666 "$BINARY_FILE_PATH" From 475ca04b2ebaa6e207e2dfc30c65d39021fb6e8c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 14 Oct 2024 16:44:20 +0200 Subject: [PATCH 40/53] Typos --- src/Analyzer/Passes/CrossToInnerJoinPass.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp index 3603745b279..03e27c9fdd2 100644 --- a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp +++ b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes namespace { -void exctractJoinConditions(const QueryTreeNodePtr & node, QueryTreeNodes & equi_conditions, QueryTreeNodes & other) +void extractJoinConditions(const QueryTreeNodePtr & node, QueryTreeNodes & equi_conditions, QueryTreeNodes & other) { auto * func = node->as(); if (!func) @@ -52,7 +52,7 @@ void exctractJoinConditions(const QueryTreeNodePtr & node, QueryTreeNodes & equi else if (func->getFunctionName() == "and") { for (const auto & arg : args) - exctractJoinConditions(arg, equi_conditions, other); + extractJoinConditions(arg, equi_conditions, other); } else { @@ -118,7 +118,7 @@ public: QueryTreeNodes equi_conditions; QueryTreeNodes other_conditions; - exctractJoinConditions(where_condition, equi_conditions, other_conditions); + extractJoinConditions(where_condition, equi_conditions, other_conditions); bool can_convert_cross_to_inner = false; for (auto & condition : equi_conditions) { From eade26985be1a85ad5d93a79d5be44e86d11b20d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 14 Oct 2024 17:20:22 +0200 Subject: [PATCH 41/53] Try fix 00180_no_seek_avoiding_when_reading_from_cache --- src/Interpreters/Cache/FileCache.cpp | 2 +- tests/config/config.d/storage_conf.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e0e92f603e7..f7b7ffc5aea 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1078,7 +1078,7 @@ void FileCache::freeSpaceRatioKeepingThreadFunc() if (eviction_candidates.size() > 0) { LOG_TRACE(log, "Current usage {}/{} in size, {}/{} in elements count " - "(trying to keep size ration at {} and elements ratio at {}). " + "(trying to keep size ratio at {} and elements ratio at {}). " "Collected {} eviction candidates, " "skipped {} candidates while iterating", main_priority->getSize(lock), size_limit, diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 091071f0637..74bad7528c8 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,7 @@ cache s3_disk s3_cache/ - 104857600 + 209715200 5Mi 1 100 From 67b92982b83d2dc416b20fe077c8045b797360a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Oct 2024 18:30:52 +0200 Subject: [PATCH 42/53] Don't do validation when synchronizing user_directories from keeper --- src/Access/AuthenticationData.cpp | 47 +++++++++++++------------ src/Access/AuthenticationData.h | 10 +++--- src/Access/UsersConfigAccessStorage.cpp | 7 ++-- 3 files changed, 34 insertions(+), 30 deletions(-) diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 92bf2145909..57a1cd756ff 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -117,20 +117,20 @@ bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs) } -void AuthenticationData::setPassword(const String & password_) +void AuthenticationData::setPassword(const String & password_, bool validate) { switch (type) { case AuthenticationType::PLAINTEXT_PASSWORD: - setPasswordHashBinary(Util::stringToDigest(password_)); + setPasswordHashBinary(Util::stringToDigest(password_), validate); return; case AuthenticationType::SHA256_PASSWORD: - setPasswordHashBinary(Util::encodeSHA256(password_)); + setPasswordHashBinary(Util::encodeSHA256(password_), validate); return; case AuthenticationType::DOUBLE_SHA1_PASSWORD: - setPasswordHashBinary(Util::encodeDoubleSHA1(password_)); + setPasswordHashBinary(Util::encodeDoubleSHA1(password_), validate); return; case AuthenticationType::BCRYPT_PASSWORD: @@ -149,12 +149,12 @@ void AuthenticationData::setPassword(const String & password_) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "setPassword(): authentication type {} not supported", toString(type)); } -void AuthenticationData::setPasswordBcrypt(const String & password_, int workfactor_) +void AuthenticationData::setPasswordBcrypt(const String & password_, int workfactor_, bool validate) { if (type != AuthenticationType::BCRYPT_PASSWORD) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify bcrypt password for authentication type {}", toString(type)); - setPasswordHashBinary(Util::encodeBcrypt(password_, workfactor_)); + setPasswordHashBinary(Util::encodeBcrypt(password_, workfactor_), validate); } String AuthenticationData::getPassword() const @@ -165,7 +165,7 @@ String AuthenticationData::getPassword() const } -void AuthenticationData::setPasswordHashHex(const String & hash) +void AuthenticationData::setPasswordHashHex(const String & hash, bool validate) { Digest digest; digest.resize(hash.size() / 2); @@ -179,7 +179,7 @@ void AuthenticationData::setPasswordHashHex(const String & hash) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read password hash in hex, check for valid characters [0-9a-fA-F] and length"); } - setPasswordHashBinary(digest); + setPasswordHashBinary(digest, validate); } @@ -195,7 +195,7 @@ String AuthenticationData::getPasswordHashHex() const } -void AuthenticationData::setPasswordHashBinary(const Digest & hash) +void AuthenticationData::setPasswordHashBinary(const Digest & hash, bool validate) { switch (type) { @@ -217,7 +217,7 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash) case AuthenticationType::DOUBLE_SHA1_PASSWORD: { - if (hash.size() != 20) + if (validate && hash.size() != 20) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password hash for the 'DOUBLE_SHA1_PASSWORD' authentication type has length {} " "but must be exactly 20 bytes.", hash.size()); @@ -231,7 +231,7 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash) /// However the library we use to encode it requires hash string to be 64 characters long, /// so we also allow the hash of this length. - if (hash.size() != 59 && hash.size() != 60 && hash.size() != 64) + if (validate && hash.size() != 59 && hash.size() != 60 && hash.size() != 64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password hash for the 'BCRYPT_PASSWORD' authentication type has length {} " "but must be 59 or 60 bytes.", hash.size()); @@ -240,10 +240,13 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash) resized.resize(64); #if USE_BCRYPT - /// Verify that it is a valid hash - int ret = bcrypt_checkpw("", reinterpret_cast(resized.data())); - if (ret == -1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not decode the provided hash with 'bcrypt_hash'"); + if (validate) + { + /// Verify that it is a valid hash + int ret = bcrypt_checkpw("", reinterpret_cast(resized.data())); + if (ret == -1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not decode the provided hash with 'bcrypt_hash'"); + } #endif password_hash = hash; @@ -385,7 +388,7 @@ std::shared_ptr AuthenticationData::toAST() const } -AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & query, ContextPtr context, bool check_password_rules) +AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & query, ContextPtr context, bool validate) { if (query.type && query.type == AuthenticationType::NO_PASSWORD) return AuthenticationData(); @@ -431,7 +434,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que if (!query.type && !context) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get default password type without context"); - if (check_password_rules && !context) + if (validate && !context) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot check password complexity rules without context"); if (query.type == AuthenticationType::BCRYPT_PASSWORD && !context) @@ -448,13 +451,13 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que AuthenticationData auth_data(current_type); - if (check_password_rules) + if (validate) context->getAccessControl().checkPasswordComplexityRules(value); if (query.type == AuthenticationType::BCRYPT_PASSWORD) { int workfactor = context->getAccessControl().getBcryptWorkfactor(); - auth_data.setPasswordBcrypt(value, workfactor); + auth_data.setPasswordBcrypt(value, workfactor, validate); return auth_data; } @@ -486,7 +489,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que #endif } - auth_data.setPassword(value); + auth_data.setPassword(value, validate); return auth_data; } @@ -498,11 +501,11 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que if (query.type == AuthenticationType::BCRYPT_PASSWORD) { - auth_data.setPasswordHashBinary(AuthenticationData::Util::stringToDigest(value)); + auth_data.setPasswordHashBinary(AuthenticationData::Util::stringToDigest(value), validate); return auth_data; } - auth_data.setPasswordHashHex(value); + auth_data.setPasswordHashHex(value, validate); if (query.type == AuthenticationType::SHA256_PASSWORD && args_size == 2) diff --git a/src/Access/AuthenticationData.h b/src/Access/AuthenticationData.h index 8093fe1d888..a0c100264f8 100644 --- a/src/Access/AuthenticationData.h +++ b/src/Access/AuthenticationData.h @@ -31,17 +31,17 @@ public: AuthenticationType getType() const { return type; } /// Sets the password and encrypt it using the authentication type set in the constructor. - void setPassword(const String & password_); + void setPassword(const String & password_, bool validate); /// Returns the password. Allowed to use only for Type::PLAINTEXT_PASSWORD. String getPassword() const; /// Sets the password as a string of hexadecimal digits. - void setPasswordHashHex(const String & hash); + void setPasswordHashHex(const String & hash, bool validate); String getPasswordHashHex() const; /// Sets the password in binary form. - void setPasswordHashBinary(const Digest & hash); + void setPasswordHashBinary(const Digest & hash, bool validate); const Digest & getPasswordHashBinary() const { return password_hash; } /// Sets the salt in String form. @@ -49,7 +49,7 @@ public: String getSalt() const; /// Sets the password using bcrypt hash with specified workfactor - void setPasswordBcrypt(const String & password_, int workfactor_); + void setPasswordBcrypt(const String & password_, int workfactor_, bool validate); /// Sets the server name for authentication type LDAP. const String & getLDAPServerName() const { return ldap_server_name; } @@ -77,7 +77,7 @@ public: friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs); friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); } - static AuthenticationData fromAST(const ASTAuthenticationData & query, ContextPtr context, bool check_password_rules); + static AuthenticationData fromAST(const ASTAuthenticationData & query, ContextPtr context, bool validate); std::shared_ptr toAST() const; struct Util diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index b89d5f136b3..eddc7ca1e0e 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -121,6 +121,7 @@ namespace bool allow_no_password, bool allow_plaintext_password) { + const bool validate = true; auto user = std::make_shared(); user->setName(user_name); String user_config = "users." + user_name; @@ -157,17 +158,17 @@ namespace if (has_password_plaintext) { user->authentication_methods.emplace_back(AuthenticationType::PLAINTEXT_PASSWORD); - user->authentication_methods.back().setPassword(config.getString(user_config + ".password")); + user->authentication_methods.back().setPassword(config.getString(user_config + ".password"), validate); } else if (has_password_sha256_hex) { user->authentication_methods.emplace_back(AuthenticationType::SHA256_PASSWORD); - user->authentication_methods.back().setPasswordHashHex(config.getString(user_config + ".password_sha256_hex")); + user->authentication_methods.back().setPasswordHashHex(config.getString(user_config + ".password_sha256_hex"), validate); } else if (has_password_double_sha1_hex) { user->authentication_methods.emplace_back(AuthenticationType::DOUBLE_SHA1_PASSWORD); - user->authentication_methods.back().setPasswordHashHex(config.getString(user_config + ".password_double_sha1_hex")); + user->authentication_methods.back().setPasswordHashHex(config.getString(user_config + ".password_double_sha1_hex"), validate); } else if (has_ldap) { From a23f941b7f22e7e03d71373725cec6f958bea3f6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 14 Oct 2024 18:44:14 +0200 Subject: [PATCH 43/53] Restrict statement_timeout --- tests/integration/compose/docker_compose_postgres.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_postgres.yml b/tests/integration/compose/docker_compose_postgres.yml index 20738ec7c6d..df6ca8b0aad 100644 --- a/tests/integration/compose/docker_compose_postgres.yml +++ b/tests/integration/compose/docker_compose_postgres.yml @@ -1,7 +1,7 @@ services: postgres1: image: postgres - command: ["postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=4", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_statement=all", "-c", "max_connections=200"] + command: ["postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=4", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_statement=all", "-c", "max_connections=200", "-c", "statement_timeout=180000"] restart: always expose: - ${POSTGRES_PORT:-5432} From 71148264eee5fb3d6e3d8f44ce5af10f67b64d7e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 14 Oct 2024 19:01:26 +0200 Subject: [PATCH 44/53] Add a comment --- tests/integration/compose/docker_compose_postgres.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_postgres.yml b/tests/integration/compose/docker_compose_postgres.yml index df6ca8b0aad..e072589ac4a 100644 --- a/tests/integration/compose/docker_compose_postgres.yml +++ b/tests/integration/compose/docker_compose_postgres.yml @@ -1,7 +1,7 @@ services: postgres1: image: postgres - command: ["postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=4", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_statement=all", "-c", "max_connections=200", "-c", "statement_timeout=180000"] + command: ["postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=4", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_statement=all", "-c", "max_connections=200", "-c", "statement_timeout=180000"] # statement_timeout: avoid test timeout because of postgres getting unavailable restart: always expose: - ${POSTGRES_PORT:-5432} From 456d92e22999eedce951c4e3087340038991b90f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 14 Oct 2024 19:58:10 +0200 Subject: [PATCH 45/53] empty commit From 36fbc4ac32cf74432afbf63c626d539630371bb0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Oct 2024 19:02:09 +0000 Subject: [PATCH 46/53] Docs: Follow-up for #70585 --- cmake/cpu_features.cmake | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index b2a648c7ca3..2bb6deb4847 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -11,6 +11,10 @@ option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries n if (ARCH_NATIVE) set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") + # Populate the ENABLE_ option flags. This is required for the build of some third-party dependencies, specifically snappy, which + # (somewhat weirdly) expects the relative SNAPPY_HAVE_ preprocessor variables to be populated, in addition to the microarchitecture + # feature flags being enabled in the compiler. This fixes the ARCH_NATIVE flag by automatically populating the ENABLE_ option flags + # according to the current CPU's capabilities, detected using clang. if (ARCH_AMD64) execute_process( COMMAND sh -c "clang -E - -march=native -###" From 81f07df2e01db3f0de8b3cab0df9f9d6d4084323 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Mon, 14 Oct 2024 12:17:14 -0700 Subject: [PATCH 47/53] [Docs] Remove cloud only badge --- docs/en/engines/table-engines/integrations/mysql.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/mysql.md b/docs/en/engines/table-engines/integrations/mysql.md index e9d0a43242b..3522bd57cc3 100644 --- a/docs/en/engines/table-engines/integrations/mysql.md +++ b/docs/en/engines/table-engines/integrations/mysql.md @@ -4,12 +4,8 @@ sidebar_position: 138 sidebar_label: MySQL --- -import CloudAvailableBadge from '@theme/badges/CloudAvailableBadge'; - # MySQL Table Engine - - The MySQL engine allows you to perform `SELECT` and `INSERT` queries on data that is stored on a remote MySQL server. ## Creating a Table {#creating-a-table} From ef9b9a237393d494160587975afe39399d4ced01 Mon Sep 17 00:00:00 2001 From: Sharath K S Date: Mon, 14 Oct 2024 13:47:04 -0700 Subject: [PATCH 48/53] Refactor executeQuery.cpp to use convertFieldToString --- src/Interpreters/executeQuery.cpp | 25 ++----------------------- 1 file changed, 2 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2418eaff5b9..723f8c4c6e0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -577,29 +578,7 @@ void logQueryFinish( for (const auto & name : changed_settings_names) { Field value = settings.get(name); - String value_str; - - switch (value.getType()) - { - case Field::Types::Which::String: - value_str = value.safeGet(); - break; - case Field::Types::Which::Int64: - value_str = std::to_string(value.safeGet()); - break; - case Field::Types::Which::Float64: - value_str = std::to_string(value.safeGet()); - break; - case Field::Types::Which::Bool: - value_str = value.safeGet() ? "true" : "false"; - break; - case Field::Types::Which::UInt64: - value_str = std::to_string(value.safeGet()); - break; - default: - value_str = "Unsupported type: " + std::to_string(static_cast(value.getType())); - break; - } + String value_str = convertFieldToString(value); query_span->addAttribute(fmt::format("clickhouse.setting.{}", name), value_str); From a906883919c757ff49913a38f25fc523e35abc46 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Oct 2024 00:51:17 +0200 Subject: [PATCH 49/53] Remove strange file --- docs/mkdocs.yml | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 docs/mkdocs.yml diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml deleted file mode 100644 index d29df2fc24b..00000000000 --- a/docs/mkdocs.yml +++ /dev/null @@ -1,2 +0,0 @@ -# Just an empty yaml file. Keep it alone. -{} From d6e546bd54c6a158cfad06afb2a7027a0e649e65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Oct 2024 00:53:31 +0200 Subject: [PATCH 50/53] Remove more files --- docs/_includes/install/deb.sh | 11 ----------- docs/_includes/install/rpm.sh | 6 ------ docs/_includes/install/tgz.sh | 32 -------------------------------- 3 files changed, 49 deletions(-) delete mode 100644 docs/_includes/install/deb.sh delete mode 100644 docs/_includes/install/rpm.sh delete mode 100644 docs/_includes/install/tgz.sh diff --git a/docs/_includes/install/deb.sh b/docs/_includes/install/deb.sh deleted file mode 100644 index 0daf12a132f..00000000000 --- a/docs/_includes/install/deb.sh +++ /dev/null @@ -1,11 +0,0 @@ -sudo apt-get install -y apt-transport-https ca-certificates dirmngr -sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 - -echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ - /etc/apt/sources.list.d/clickhouse.list -sudo apt-get update - -sudo apt-get install -y clickhouse-server clickhouse-client - -sudo service clickhouse-server start -clickhouse-client # or "clickhouse-client --password" if you've set up a password. diff --git a/docs/_includes/install/rpm.sh b/docs/_includes/install/rpm.sh deleted file mode 100644 index ff99018f872..00000000000 --- a/docs/_includes/install/rpm.sh +++ /dev/null @@ -1,6 +0,0 @@ -sudo yum install -y yum-utils -sudo yum-config-manager --add-repo https://packages.clickhouse.com/rpm/clickhouse.repo -sudo yum install -y clickhouse-server clickhouse-client - -sudo /etc/init.d/clickhouse-server start -clickhouse-client # or "clickhouse-client --password" if you set up a password. diff --git a/docs/_includes/install/tgz.sh b/docs/_includes/install/tgz.sh deleted file mode 100644 index e42353f7b48..00000000000 --- a/docs/_includes/install/tgz.sh +++ /dev/null @@ -1,32 +0,0 @@ -LATEST_VERSION=$(curl -s https://packages.clickhouse.com/tgz/stable/ | \ - grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | sort -V -r | head -n 1) -export LATEST_VERSION - -case $(uname -m) in - x86_64) ARCH=amd64 ;; - aarch64) ARCH=arm64 ;; - *) echo "Unknown architecture $(uname -m)"; exit 1 ;; -esac - -for PKG in clickhouse-common-static clickhouse-common-static-dbg clickhouse-server clickhouse-client -do - curl -fO "https://packages.clickhouse.com/tgz/stable/$PKG-$LATEST_VERSION-${ARCH}.tgz" \ - || curl -fO "https://packages.clickhouse.com/tgz/stable/$PKG-$LATEST_VERSION.tgz" -done - -tar -xzvf "clickhouse-common-static-$LATEST_VERSION-${ARCH}.tgz" \ - || tar -xzvf "clickhouse-common-static-$LATEST_VERSION.tgz" -sudo "clickhouse-common-static-$LATEST_VERSION/install/doinst.sh" - -tar -xzvf "clickhouse-common-static-dbg-$LATEST_VERSION-${ARCH}.tgz" \ - || tar -xzvf "clickhouse-common-static-dbg-$LATEST_VERSION.tgz" -sudo "clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh" - -tar -xzvf "clickhouse-server-$LATEST_VERSION-${ARCH}.tgz" \ - || tar -xzvf "clickhouse-server-$LATEST_VERSION.tgz" -sudo "clickhouse-server-$LATEST_VERSION/install/doinst.sh" configure -sudo /etc/init.d/clickhouse-server start - -tar -xzvf "clickhouse-client-$LATEST_VERSION-${ARCH}.tgz" \ - || tar -xzvf "clickhouse-client-$LATEST_VERSION.tgz" -sudo "clickhouse-client-$LATEST_VERSION/install/doinst.sh" From 13d74479b9c7bd8f04b39d6379572df15f0c8184 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Oct 2024 00:54:26 +0200 Subject: [PATCH 51/53] Remove more files --- docs/README.md | 1 - .../template-setting.md | 27 ------------------- 2 files changed, 28 deletions(-) delete mode 100644 docs/_description_templates/template-setting.md diff --git a/docs/README.md b/docs/README.md index d1260312166..42fdda88732 100644 --- a/docs/README.md +++ b/docs/README.md @@ -196,7 +196,6 @@ When writing docs, you can use prepared templates. Copy the code of a template a Templates: - [Function](_description_templates/template-function.md) -- [Setting](_description_templates/template-setting.md) - [Server Setting](_description_templates/template-server-setting.md) - [Database or Table engine](_description_templates/template-engine.md) - [System table](_description_templates/template-system-table.md) diff --git a/docs/_description_templates/template-setting.md b/docs/_description_templates/template-setting.md deleted file mode 100644 index f4525d872df..00000000000 --- a/docs/_description_templates/template-setting.md +++ /dev/null @@ -1,27 +0,0 @@ -## setting_name {#setting_name} - -Description. - -For the switch setting, use the typical phrase: “Enables or disables something ...”. - -Possible values: - -*For switcher setting:* - -- 0 — Disabled. -- 1 — Enabled. - -*For another setting (typical phrases):* - -- Positive integer. -- 0 — Disabled or unlimited or something else. - -Default value: `value`. - -**Additional Info** (Optional) - -The name of an additional section can be any, for example, **Usage**. - -**See Also** (Optional) - -- [link](#) From 80bd0cd58d31e9c6668778c6756984de3c79c97c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Oct 2024 00:56:27 +0200 Subject: [PATCH 52/53] Even more files are removed --- docs/en/sql-reference/ansi.md | 190 ------------------ docs/ru/introduction/distinctive-features.md | 2 +- docs/ru/sql-reference/ansi.mdx | 10 - docs/zh/introduction/distinctive-features.md | 2 +- docs/zh/sql-reference/ansi.md | 191 ------------------- 5 files changed, 2 insertions(+), 393 deletions(-) delete mode 100644 docs/en/sql-reference/ansi.md delete mode 100644 docs/zh/sql-reference/ansi.md diff --git a/docs/en/sql-reference/ansi.md b/docs/en/sql-reference/ansi.md deleted file mode 100644 index 6ba7b16831e..00000000000 --- a/docs/en/sql-reference/ansi.md +++ /dev/null @@ -1,190 +0,0 @@ ---- -slug: /en/sql-reference/ansi -sidebar_position: 40 -sidebar_label: ANSI Compatibility -title: "ANSI SQL Compatibility of ClickHouse SQL Dialect" ---- - -:::note -This article relies on Table 38, “Feature taxonomy and definition for mandatory features”, Annex F of [ISO/IEC CD 9075-2:2011](https://www.iso.org/obp/ui/#iso:std:iso-iec:9075:-2:ed-4:v1:en:sec:8). -::: - -## Differences in Behaviour - -The following table lists cases when query feature works in ClickHouse, but behaves not as specified in ANSI SQL. - -| Feature ID | Feature Name | Difference | -|------------|-----------------------------|-----------------------------------------------------------------------------------------------------------| -| E011 | Numeric data types | Numeric literal with period is interpreted as approximate (`Float64`) instead of exact (`Decimal`) | -| E051-05 | Select items can be renamed | Item renames have a wider visibility scope than just the SELECT result | -| E141-01 | NOT NULL constraints | `NOT NULL` is implied for table columns by default | -| E011-04 | Arithmetic operators | ClickHouse overflows instead of checked arithmetic and changes the result data type based on custom rules | - -## Feature Status - -| Feature ID | Feature Name | Status | Comment | -|------------|--------------------------------------------------------------------------------------------------------------------------|----------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| **E011** | **Numeric data types** | Partial | | -| E011-01 | INTEGER and SMALLINT data types | Yes | | -| E011-02 | REAL, DOUBLE PRECISION and FLOAT data types data types | Yes | | -| E011-03 | DECIMAL and NUMERIC data types | Yes | | -| E011-04 | Arithmetic operators | Yes | | -| E011-05 | Numeric comparison | Yes | | -| E011-06 | Implicit casting among the numeric data types | No | ANSI SQL allows arbitrary implicit cast between numeric types, while ClickHouse relies on functions having multiple overloads instead of implicit cast | -| **E021** | **Character string types** | Partial | | -| E021-01 | CHARACTER data type | Yes | | -| E021-02 | CHARACTER VARYING data type | Yes | | -| E021-03 | Character literals | Yes | | -| E021-04 | CHARACTER_LENGTH function | Partial | No `USING` clause | -| E021-05 | OCTET_LENGTH function | No | `LENGTH` behaves similarly | -| E021-06 | SUBSTRING | Partial | No support for `SIMILAR` and `ESCAPE` clauses, no `SUBSTRING_REGEX` variant | -| E021-07 | Character concatenation | Partial | No `COLLATE` clause | -| E021-08 | UPPER and LOWER functions | Yes | | -| E021-09 | TRIM function | Yes | | -| E021-10 | Implicit casting among the fixed-length and variable-length character string types | Partial | ANSI SQL allows arbitrary implicit cast between string types, while ClickHouse relies on functions having multiple overloads instead of implicit cast | -| E021-11 | POSITION function | Partial | No support for `IN` and `USING` clauses, no `POSITION_REGEX` variant | -| E021-12 | Character comparison | Yes | | -| **E031** | **Identifiers** | Partial| | -| E031-01 | Delimited identifiers | Partial | Unicode literal support is limited | -| E031-02 | Lower case identifiers | Yes | | -| E031-03 | Trailing underscore | Yes | | -| **E051** | **Basic query specification** | Partial| | -| E051-01 | SELECT DISTINCT | Yes | | -| E051-02 | GROUP BY clause | Yes | | -| E051-04 | GROUP BY can contain columns not in ``中出现的列 | 是 {.text-success} | | -| E051-05 | SELECT 的列可以重命名 | 是 {.text-success} | | -| E051-06 | HAVING 从句 | 是 {.text-success} | | -| E051-07 | SELECT 选择的列中允许出现\* | 是 {.text-success} | | -| E051-08 | FROM 从句中的关联名称 | 是 {.text-success} | | -| E051-09 | 重命名 FROM 从句中的列 | 否 {.text-danger} | | -| **E061** | **基本谓词和搜索条件** | **部分**{.text-warning} | | -| E061-01 | 比较谓词 | 是 {.text-success} | | -| E061-02 | BETWEEN 谓词 | 部分 {.text-warning} | 不支持 `SYMMETRIC` 和 `ASYMMETRIC` 从句 | -| E061-03 | IN 谓词后可接值列表 | 是 {.text-success} | | -| E061-04 | LIKE 谓词 | 是 {.text-success} | | -| E061-05 | LIKE 谓词后接 ESCAPE 从句 | 否 {.text-danger} | | -| E061-06 | NULL 谓词 | 是 {.text-success} | | -| E061-07 | 量化比较谓词(ALL、SOME、ANY) | 否 {.text-danger} | | -| E061-08 | EXISTS 谓词 | 否 {.text-danger} | | -| E061-09 | 比较谓词中的子查询 | 是 {.text-success} | | -| E061-11 | IN 谓词中的子查询 | 是 {.text-success} | | -| E061-12 | 量化比较谓词(BETWEEN、IN、LIKE)中的子查询 | 否 {.text-danger} | | -| E061-13 | 相关子查询 | 否 {.text-danger} | | -| E061-14 | 搜索条件 | 是 {.text-success} | | -| **E071** | **基本查询表达式** | **部分**{.text-warning} | | -| E071-01 | UNION DISTINCT 表运算符 | 是 {.text-success} | | -| E071-02 | UNION ALL 表运算符 | 是 {.text-success} | | -| E071-03 | EXCEPT DISTINCT 表运算符 | 否 {.text-danger} | | -| E071-05 | 通过表运算符组合的列不必具有完全相同的数据类型 | 是 {.text-success} | | -| E071-06 | 子查询中的表运算符 | 是 {.text-success} | | -| **E081** | **基本权限** | **是**{.text-success} | | -| E081-01 | 表级别的SELECT(查询)权限 | 是 {.text-success} | | -| E081-02 | DELETE(删除)权限 | 是 {.text-success} | | -| E081-03 | 表级别的INSERT(插入)权限 | 是 {.text-success} | | -| E081-04 | 表级别的UPDATE(更新)权限 | 是 {.text-success} | | -| E081-05 | 列级别的UPDATE(更新)权限 | 是 {.text-success} | | -| E081-06 | 表级别的REFERENCES(引用)权限 | 是 {.text-success} | | -| E081-07 | 列级别的REFERENCES(引用)权限 | 是 {.text-success} | | -| E081-08 | WITH GRANT OPTION | 是 {.text-success} | | -| E081-09 | USAGE(使用)权限 | 是 {.text-success} | | -| E081-10 | EXECUTE(执行)权限 | 是 {.text-success} | | -| **E091** | **集合函数** | **是**{.text-success} | | -| E091-01 | AVG | 是 {.text-success} | | -| E091-02 | COUNT | 是 {.text-success} | | -| E091-03 | MAX | 是 {.text-success} | | -| E091-04 | MIN | 是 {.text-success} | | -| E091-05 | SUM | 是 {.text-success} | | -| E091-06 | ALL修饰词 | 否。 {.text-danger} | | -| E091-07 | DISTINCT修饰词 | 是 {.text-success} | 并非所有聚合函数都支持该修饰词 | -| **E101** | **基本数据操作** | **部分**{.text-warning} | | -| E101-01 | INSERT(插入)语句 | 是 {.text-success} | 注:ClickHouse中的主键并不隐含`UNIQUE` 约束 | -| E101-03 | 可指定范围的UPDATE(更新)语句 | 部分 {.text-warning} | `ALTER UPDATE` 语句用来批量更新数据 | -| E101-04 | 可指定范围的DELETE(删除)语句 | 部分 {.text-warning} | `ALTER DELETE` 语句用来批量删除数据 | -| **E111** | **返回一行的SELECT语句** | **否**{.text-danger} | | -| **E121** | **基本游标支持** | **否**{.text-danger} | | -| E121-01 | DECLARE CURSOR | 否 {.text-danger} | | -| E121-02 | ORDER BY 涉及的列不需要出现在SELECT的列中 | 是 {.text-success} | | -| E121-03 | ORDER BY 从句中的表达式 | 是 {.text-success} | | -| E121-04 | OPEN 语句 | 否 {.text-danger} | | -| E121-06 | 受游标位置控制的 UPDATE 语句 | 否 {.text-danger} | | -| E121-07 | 受游标位置控制的 DELETE 语句 | 否 {.text-danger} | | -| E121-08 | CLOSE 语句 | 否 {.text-danger} | | -| E121-10 | FETCH 语句中包含隐式NEXT | 否 {.text-danger} | | -| E121-17 | WITH HOLD 游标 | 否 {.text-danger} | | -| **E131** | **空值支持** | **是**{.text-success} | 有部分限制 | -| **E141** | **基本完整性约束** | **部分**{.text-warning} | | -| E141-01 | NOT NULL(非空)约束 | 是 {.text-success} | 注: 默认情况下ClickHouse表中的列隐含`NOT NULL`约束 | -| E141-02 | NOT NULL(非空)列的UNIQUE(唯一)约束 | 否 {.text-danger} | | -| E141-03 | 主键约束 | 部分 {.text-warning} | | -| E141-04 | 对于引用删除和引用更新操作,基本的FOREIGN KEY(外键)约束默认不进行任何操作(NO ACTION) | 否 {.text-danger} | | -| E141-06 | CHECK(检查)约束 | 是 {.text-success} | | -| E141-07 | 列默认值 | 是 {.text-success} | | -| E141-08 | 在主键上推断非空 | 是 {.text-success} | | -| E141-10 | 可以按任何顺序指定外键中的名称 | 否 {.text-danger} | | -| **E151** | **事务支持** | **否**{.text-danger} | | -| E151-01 | COMMIT(提交)语句 | 否 {.text-danger} | | -| E151-02 | ROLLBACK(回滚)语句 | 否 {.text-danger} | | -| **E152** | **基本的SET TRANSACTION(设置事务隔离级别)语句** | **否**{.text-danger} | | -| E152-01 | SET TRANSACTION语句:ISOLATION LEVEL SERIALIZABLE(隔离级别为串行化)从句 | 否 {.text-danger} | | -| E152-02 | SET TRANSACTION语句:READ ONLY(只读)和READ WRITE(读写)从句 | 否 {.text-danger} | | -| **E153** | **具有子查询的可更新查询** | **是**{.text-success} | | -| **E161** | **使用“--”符号作为SQL注释** | **是**{.text-success} | | -| **E171** | **SQLSTATE支持** | **否**{.text-danger} | | -| **E182** | **主机语言绑定** | **否**{.text-danger} | | -| **F031** | **基本架构操作** | **部分**{.text-warning} | | -| F031-01 | 使用 CREATE TABLE 语句创建持久表 | 部分 {.text-warning} | 不支持 `SYSTEM VERSIONING`, `ON COMMIT`, `GLOBAL`, `LOCAL`, `PRESERVE`, `DELETE`, `REF IS`, `WITH OPTIONS`, `UNDER`, `LIKE`, `PERIOD FOR` 从句,不支持用户解析的数据类型 | -| F031-02 | CREATE VIEW(创建视图)语句 | 部分 {.text-warning} | 不支持 `RECURSIVE`, `CHECK`, `UNDER`, `WITH OPTIONS` 从句,不支持用户解析的数据类型 | -| F031-03 | GRANT(授权)语句 | 是 {.text-success} | | -| F031-04 | ALTER TABLE语句:ADD COLUMN从句 | 是 {.text-success} | 不支持 `GENERATED` 从句和以系统时间做参数 | -| F031-13 | DROP TABLE语句:RESTRICT从句 | 否 {.text-danger} | | -| F031-16 | DROP VIEW语句:RESTRICT子句 | 否 {.text-danger} | | -| F031-19 | REVOKE语句:RESTRICT子句 | 否 {.text-danger} | | -| **F041** | **基本连接关系** | **部分**{.text-warning} | | -| F041-01 | Inner join(但不一定是INNER关键字) | 是 {.text-success} | | -| F041-02 | INNER 关键字 | 是 {.text-success} | | -| F041-03 | LEFT OUTER JOIN | 是 {.text-success} | | -| F041-04 | RIGHT OUTER JOIN | 是 {.text-success} | | -| F041-05 | 外连接可嵌套 | 是 {.text-success} | | -| F041-07 | 左外部连接或右外连接中的内部表也可用于内部联接 | 是 {.text-success} | | -| F041-08 | 支持所有比较运算符(而不仅仅是=) | 否 {.text-danger} | | -| **F051** | **基本日期和时间** | **部分**{.text-warning} | | -| F051-01 | DATE(日期)数据类型(并支持用于表达日期的字面量) | 是 {.text-success} | | -| F051-02 | TIME(时间)数据类型(并支持用于表达时间的字面量),小数秒精度至少为0 | 否 {.text-danger} | | -| F051-03 | 时间戳数据类型(并支持用于表达时间戳的字面量),小数秒精度至少为0和6 | 是 {.text-danger} | | -| F051-04 | 日期、时间和时间戳数据类型的比较谓词 | 是 {.text-success} | | -| F051-05 | DateTime 类型和字符串形式表达的时间之间的显式转换 | 是 {.text-success} | | -| F051-06 | CURRENT_DATE | 否 {.text-danger} | 使用`today()`替代 | -| F051-07 | LOCALTIME | 否 {.text-danger} | 使用`now()`替代 | -| F051-08 | LOCALTIMESTAMP | 否 {.text-danger} | | -| **F081** | **视图的UNION和EXCEPT操作** | **部分**{.text-warning} | | -| **F131** | **分组操作** | **部分**{.text-warning} | | -| F131-01 | 在具有分组视图的查询中支持 WHERE、GROUP BY 和 HAVING 子句 | 是 {.text-success} | | -| F131-02 | 在分组视图中支持多张表 | 是 {.text-success} | | -| F131-03 | 分组视图的查询中支持集合函数 | 是 {.text-success} | | -| F131-04 | 带有 `GROUP BY` 和 `HAVING` 从句,以及分组视图的子查询 | 是 {.text-success} | | -| F131-05 | 带有 `GROUP BY` 和 `HAVING` 从句,以及分组视图的仅返回1条记录的SELECT查询 | 否 {.text-danger} | | -| **F181** | **多模块支持** | **否**{.text-danger} | | -| **F201** | **CAST 函数** | **是**{.text-success} | | -| **F221** | **显式默认值** | **否**{.text-danger} | | -| **F261** | **CASE 表达式** | **是**{.text-success} | | -| F261-01 | 简单 CASE 表达式 | 是 {.text-success} | | -| F261-02 | 搜索型 CASE 表达式 | 是 {.text-success} | | -| F261-03 | NULLIF | 是 {.text-success} | | -| F261-04 | COALESCE | 是 {.text-success} | | -| **F311** | **架构定义语句** | **部分**{.text-warning} | | -| F311-01 | CREATE SCHEMA | 部分 {.text-warning} | 见`CREATE DATABASE` | -| F311-02 | 用于创建持久表的 CREATE TABLE | 是 {.text-success} | | -| F311-03 | CREATE VIEW | 是 {.text-success} | | -| F311-04 | CREATE VIEW: WITH CHECK OPTION | 否 {.text-danger} | | -| F311-05 | GRANT 语句 | 是 {.text-success} | | -| **F471** | **标量子查询** | **是**{.text-success} | | -| **F481** | **扩展 NULL 谓词** | **是**{.text-success} | | -| **F812** | **基本标志位** | **否**{.text-danger} | -| **S011** | **用于不重复数据的数据类型** | **否**{.text-danger} | -| **T321** | **基本的SQL调用例程** | **否**{.text-danger} | | -| T321-01 | 没有重载的用户定义函数 | 否{.text-danger} | | -| T321-02 | 没有重载的用户定义存储过程 | 否{.text-danger} | | -| T321-03 | 功能调用 | 否 {.text-danger} | | -| T321-04 | CALL 语句 | 否 {.text-danger} | | -| T321-05 | RETURN 语句 | 否 {.text-danger} | | -| **T631** | **IN 谓词后接一个列表** | **是**{.text-success} | | From 6f170b0948ed6068a45eccbad0f6278088523be0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Oct 2024 15:15:15 +0200 Subject: [PATCH 53/53] Empty commit