From fd271f76d118f1ef824cfa97f10262292d65c244 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 2 Dec 2025 15:30:59 +0100 Subject: [PATCH 01/49] Advertise host from config for replicated databases In containerized/cloud environments, the system hostname (e.g., pod-abc123) is not the network-accessible address. This causes replica communication to fail. Use the explicitly configured interserver_http_host from config instead of system hostname. The hostname is part of the Host ID hostname:port:database_uuid and replicated database uses the Host ID for Zookeeper registration, replica discovery and DDL coordination. Co-authored-by: Kevin Michel --- src/Databases/DatabaseReplicated.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f99884741315..7dc71990c43b 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -123,7 +123,8 @@ ZooKeeperPtr DatabaseReplicated::getZooKeeper() const static inline String getHostID(ContextPtr global_context, const UUID & db_uuid, bool secure) { UInt16 port = secure ? global_context->getTCPPortSecure().value_or(DBMS_DEFAULT_SECURE_PORT) : global_context->getTCPPort(); - return Cluster::Address::toString(getFQDNOrHostName(), port) + ':' + toString(db_uuid); + const auto host = global_context->getInterserverIOAddress().first; + return Cluster::Address::toString(host, port) + ':' + toString(db_uuid); } static inline UInt64 getMetadataHash(const String & table_name, const String & metadata) From d2853b923074ffbf4da5df9546751434812b6479 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Thu, 4 Dec 2025 15:28:05 +0100 Subject: [PATCH 02/49] Enable internal replication for DatabaseReplicated clusters MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Enable internal_replication=true for clusters created programmatically by DatabaseReplicated to improve performance and consistency of Distributed tables over Replicated databases. Unlike statically configured clusters in config.xml, DatabaseReplicated creates clusters dynamically at runtime. This change ensures these programmatically created clusters also benefit from internal replication. Changes: - Add internal_replication field to ClusterConnectionParameters struct - Set internal_replication=true when creating cluster in DatabaseReplicated - Update both Cluster constructors to pass internal_replication to addShard() This ensures that Distributed tables over Replicated databases write to one replica per shard, allowing ReplicatedMergeTree to handle replication asynchronously, instead of writing to all replicas directly. Benefits: - Reduces network traffic (1× instead of N× writes per shard) - Improves performance (one replica processes INSERT, others replicate async) - Better consistency (uses ReplicatedMergeTree replication mechanism) Co-authored-by: Kevin Michel --- src/Databases/DatabaseReplicated.cpp | 4 +++- src/Interpreters/Cluster.cpp | 20 ++++++++++++++++---- src/Interpreters/Cluster.h | 3 ++- 3 files changed, 21 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 7dc71990c43b..1bd504540e17 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -387,7 +387,9 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const /* bind_host= */ "", Priority{1}, cluster_name, - cluster_auth_info.cluster_secret}; + cluster_auth_info.cluster_secret, + /* internal_replication= */ true + }; return std::make_shared(getContext()->getSettingsRef(), shards, params); } diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index a0e4e4999b72..ada945743e81 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -594,8 +594,8 @@ Cluster::Cluster( current.size() + 1); addresses_with_failover.emplace_back(current); - - addShard(settings, std::move(current), params.treat_local_as_remote, current_shard_num); + + addShard(settings, std::move(current), params.treat_local_as_remote, current_shard_num, /* current_shard_name= */ "", /* weight= */ 1, /* insert_paths= */ {}, params.internal_replication); ++current_shard_num; } @@ -622,8 +622,8 @@ Cluster::Cluster( current.size() + 1); addresses_with_failover.emplace_back(current); - - addShard(settings, std::move(current), params.treat_local_as_remote, current_shard_num); + + addShard(settings, std::move(current), params.treat_local_as_remote, current_shard_num, /* current_shard_name= */ "", /* weight= */ 1, /* insert_paths= */ {}, params.internal_replication); ++current_shard_num; } @@ -645,6 +645,10 @@ void Cluster::addShard( ConnectionPoolPtrs all_replicas_pools; all_replicas_pools.reserve(addresses.size()); + /// "_all_replicas" is a marker that will be replaced with all replicas + /// (for creating connections in the Distributed engine) + insert_paths.compact = fmt::format("shard{}_all_replicas", current_shard_num); + for (const auto & replica : addresses) { auto replica_pool = ConnectionPoolFactory::instance().get( @@ -668,6 +672,14 @@ void Cluster::addShard( all_replicas_pools.emplace_back(replica_pool); if (replica.is_local && !treat_local_as_remote) shard_local_addresses.push_back(replica); + + if (internal_replication) + { + auto dir_name = replica.toFullString(/* use_compact_format= */ false); + if (!replica.is_local) + concatInsertPath(insert_paths.prefer_localhost_replica, dir_name); + concatInsertPath(insert_paths.no_prefer_localhost_replica, dir_name); + } } ConnectionPoolWithFailoverPtr shard_pool = std::make_shared( all_replicas_pools, diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index b5a4c51c11db..7edf2f0b03b6 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -49,6 +49,7 @@ struct ClusterConnectionParameters Priority priority{1}; String cluster_name; String cluster_secret; + bool internal_replication{false}; }; /// Cluster contains connection pools to each node @@ -69,7 +70,7 @@ class Cluster /// 'clickhouse_port' - port that this server instance listen for queries. /// This parameter is needed only to check that some address is local (points to ourself). /// - /// Used for remote() function. + /// Used for remote() function and Replicated database cluster. Cluster( const Settings & settings, const std::vector> & names, From a2d31afa4ac9ca82a47490a18cec6520c92ce441 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Thu, 4 Dec 2025 16:58:24 +0100 Subject: [PATCH 03/49] Enable ALTER DATABASE MODIFY SETTING for Replicated databases Add support for altering DatabaseReplicated settings at runtime without requiring database recreation. This is particularly important for rotating cluster_secret, which previously required dropping and recreating the entire database. Changes: - Override applySettingsChanges() in DatabaseReplicated to handle MODIFY SETTING commands - Add applyChange() and has() methods to DatabaseReplicatedSettings wrapper to expose BaseSettings functionality - Invalidate cached cluster when cluster_secret is changed to ensure new authentication credentials are used Implementation details: - Thread-safe: protected by DatabaseReplicated::mutex - Validates setting existence before applying changes - Automatically resets cluster cache when cluster_secret changes - Supports all DatabaseReplicatedSettings (max_broken_tables_ratio, max_replication_lag_to_enqueue, collection_name, etc.) Usage: ALTER DATABASE MODIFY SETTING cluster_secret='new_secret' Co-authored-by: Kevin Michel --- src/Databases/DatabaseReplicated.cpp | 14 ++++++++++++++ src/Databases/DatabaseReplicated.h | 2 ++ src/Databases/DatabaseReplicatedSettings.cpp | 10 ++++++++++ src/Databases/DatabaseReplicatedSettings.h | 3 +++ 4 files changed, 29 insertions(+) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1bd504540e17..5bd167ae0ea5 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -189,6 +189,20 @@ String DatabaseReplicated::getFullReplicaName(const String & shard, const String return shard + '|' + replica; } +void DatabaseReplicated::applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr) +{ + std::lock_guard lock{mutex}; + + for (const auto & change : settings_changes) + { + if (!db_settings.has(change.name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.name); + if (change.name == "cluster_secret") + cluster.reset(); + db_settings.applyChange(change); + } +} + String DatabaseReplicated::getFullReplicaName() const { return getFullReplicaName(shard_name, replica_name); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 0100723e28c5..917d7e0e6c63 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -52,6 +52,8 @@ class DatabaseReplicated : public DatabaseAtomic ~DatabaseReplicated() override; + void applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr query_context) override; + String getEngineName() const override { return "Replicated"; } /// If current query is initial, then the following methods add metadata updating ZooKeeper operations to current ZooKeeperMetadataTransaction. diff --git a/src/Databases/DatabaseReplicatedSettings.cpp b/src/Databases/DatabaseReplicatedSettings.cpp index c1a737af49af..14eca24b3569 100644 --- a/src/Databases/DatabaseReplicatedSettings.cpp +++ b/src/Databases/DatabaseReplicatedSettings.cpp @@ -93,6 +93,16 @@ void DatabaseReplicatedSettings::loadFromConfig(const String & config_elem, cons } } +void DatabaseReplicatedSettings::applyChange(const SettingChange & change) +{ + impl->applyChange(change); +} + +bool DatabaseReplicatedSettings::has(const String & name) const +{ + return impl->has(name); +} + String DatabaseReplicatedSettings::toString() const { return impl->toString(); diff --git a/src/Databases/DatabaseReplicatedSettings.h b/src/Databases/DatabaseReplicatedSettings.h index 4a337633f1fb..07315e14e7ce 100644 --- a/src/Databases/DatabaseReplicatedSettings.h +++ b/src/Databases/DatabaseReplicatedSettings.h @@ -2,6 +2,7 @@ #include #include +#include namespace Poco @@ -37,6 +38,8 @@ struct DatabaseReplicatedSettings void loadFromQuery(ASTStorage & storage_def); void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); + void applyChange(const SettingChange & change); + bool has(const String & name) const; String toString() const; From 609861c93eb86b92c289b685e6bdfedf4a77f830 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Fri, 5 Dec 2025 17:43:11 +0100 Subject: [PATCH 04/49] Replace MergeTree with ReplicatedMergeTree in Replicated databases Automatically rewrite non-replicated MergeTree engine names to their Replicated equivalents when creating tables in DatabaseReplicated databases. This ensures all tables in a replicated database are properly replicated, even if users specify ENGINE = MergeTree. Changes: - Add rewriteUnreplicatedMergeTreeEngines() method to StorageFactory - Call rewrite method before engine name extraction in StorageFactory::get() - Rewrite applies only on secondary queries (replica execution) - Supports all MergeTree variants (ReplacingMergeTree, SummingMergeTree, etc.) Implementation details: - Checks if engine name ends with MergeTree (catches all variants) - Verifies engine is not already replicated (starts with Replicated) - Only rewrites when query_kind is SECONDARY_QUERY and database engine is Replicated - Modifies AST in-place before storage engine instantiation This allows users to write ENGINE = MergeTree in replicated databases and automatically get ReplicatedMergeTree behavior without manual engine name specification. Co-authored-by: Kevin Michel Co-authored-by: Joe Lynch Co-authored-by: Dmitry Potepalov --- src/Storages/StorageFactory.cpp | 24 ++++++++++++++++++++++++ src/Storages/StorageFactory.h | 6 ++++++ 2 files changed, 30 insertions(+) diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 7864cb31cd82..8c4c72959968 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -8,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -124,6 +127,7 @@ StoragePtr StorageFactory::get( if (!storage_def->engine) throw Exception(ErrorCodes::ENGINE_REQUIRED, "Incorrect CREATE query: ENGINE required"); + rewriteUnreplicatedMergeTreeEngines(query.getDatabase(), local_context, storage_def->engine->name); const ASTFunction & engine_def = *storage_def->engine; if (engine_def.parameters) @@ -279,4 +283,24 @@ const StorageFactory::StorageFeatures & StorageFactory::getStorageFeatures(const return it->second.features; } +void StorageFactory::rewriteUnreplicatedMergeTreeEngines( + const String& database_name, + const ContextMutablePtr& local_context, + String & engine_name) const +{ + bool is_merge_tree_engine = endsWith(engine_name, "MergeTree"); + bool is_replicated_engine = startsWith(engine_name, "Replicated"); + + if (!is_merge_tree_engine || is_replicated_engine) + return; + + bool is_replicated_database = local_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && + DatabaseCatalog::instance().getDatabase(database_name)->getEngineName() == "Replicated"; + + if (is_replicated_database && !is_replicated_engine) + { + engine_name.insert(0, "Replicated"); + } +} + } diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index d3e4ccc969d4..ed3311b4e00c 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -142,6 +142,12 @@ class StorageFactory : private boost::noncopyable, public IHints<> private: Storages storages; + + void rewriteUnreplicatedMergeTreeEngines( + const String& database_name, + const ContextMutablePtr& local_context, + String & engine_name + ) const; }; void checkAllTypesAreAllowedInTable(const NamesAndTypesList & names_and_types); From 9aa01b8ef8b43dcdfa912107d168836ed9d5720b Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sat, 6 Dec 2025 12:54:17 +0100 Subject: [PATCH 05/49] Tolerate ZooKeeper restart with increased retries and exponential backoff When ZooKeeper restarts (especially during version upgrades), it can be unavailable for approximately 6 seconds. ClickHouse previously failed queries if ZooKeeper was not available within ~3 seconds, leading to inconsistent database state because DDL operations are not fully atomic. This change improves ZooKeeper connection resilience by: - Increasing minimum retry attempts from 3 to 6 - Adding exponential backoff between retry attempts (100ms, 200ms, 400ms...) - Capping maximum backoff at 10 seconds to prevent excessive delays The total retry window now covers typical ZooKeeper restart times (~6 seconds), allowing ClickHouse to successfully reconnect after ZooKeeper restarts without requiring manual intervention. This is particularly important during version upgrades when ZooKeeper nodes restart sequentially, as it prevents DDL operations from failing mid-execution and leaving the database in an inconsistent state. Co-authored-by: Kevin Michel --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index d7dd7013d86e..e1bde4de2337 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -490,8 +491,13 @@ void ZooKeeper::connect( if (nodes.empty()) throw Exception::fromMessage(Error::ZBADARGUMENTS, "No nodes passed to ZooKeeper constructor"); + // Ensure at least 6 attempts even if config sets fewer retries + static constexpr size_t min_num_tries = 6; + // Exponential backoff can grow very large, causing excessive delays. + static constexpr size_t max_backoff_ms = 10000; /// We always have at least one attempt to connect. - size_t num_tries = args.num_connection_retries + 1; + size_t num_tries = std::max(min_num_tries, args.num_connection_retries + 1); + size_t milliseconds_to_wait = 100; bool connected = false; bool dns_error = false; @@ -599,6 +605,8 @@ void ZooKeeper::connect( { fail_reasons << "\n" << getCurrentExceptionMessage(false) << ", " << node.address->toString(); cancelWriteBuffer(); + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait = std::min(milliseconds_to_wait * 2, max_backoff_ms); } } From 68cc3fed9c3507749ef01c58bb3381ef1c1f002e Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sun, 7 Dec 2025 10:15:54 +0100 Subject: [PATCH 06/49] Fix ClickHouse restart with replicated tables containing {shard} macro When ClickHouse restarts, tables in DatabaseReplicated databases are loaded from metadata files using ATTACH operations. If a table's ZooKeeper path contains the {shard} macro (e.g., /clickhouse/tables/{uuid}/{shard}), the macro expansion fails during server startup because shard and replica information is not populated in the MacroExpansionInfo structure. The issue occurs because is_replicated_database is only true for SECONDARY_QUERY operations (queries executed from the replicated queries log in ZooKeeper). During server restart, ATTACH operations are not SECONDARY_QUERY operations, so is_replicated_database is false, and the shard/replica information is not populated, causing the error: Code: 62. DB::Exception: No macro 'shard' in config while processing substitutions in '/clickhouse/tables/{uuid}/{shard}' This fix adds query.attach as an additional condition to populate shard and replica information during ATTACH operations, ensuring that macro expansion succeeds during server startup while maintaining the existing behavior for replication queries. The change is safe because: - query.attach is only true during server startup when loading tables from disk metadata - getReplicatedDatabaseShardName() and getReplicatedDatabaseReplicaName() use assert_cast, so they will fail fast if called on a non-replicated database - This pattern is already used elsewhere in the codebase (e.g., line 28 for allow_uuid_macro) This restores functionality that previously worked in ClickHouse 21.x with the custom {shard_name} macro, which was removed in later versions. Co-authored-by: Kevin Michel --- src/Storages/TableZnodeInfo.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/TableZnodeInfo.cpp b/src/Storages/TableZnodeInfo.cpp index b829b25dfeb3..00b612c781a5 100644 --- a/src/Storages/TableZnodeInfo.cpp +++ b/src/Storages/TableZnodeInfo.cpp @@ -55,7 +55,7 @@ TableZnodeInfo TableZnodeInfo::resolve(const String & requested_path, const Stri /// to make possible copying metadata files between replicas. Macros::MacroExpansionInfo info; info.table_id = table_id; - if (is_replicated_database) + if (is_replicated_database || query.attach) { auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); info.shard = getReplicatedDatabaseShardName(database); From 7ce073c283a2cc8b617e4b9a8f19e2c7a5da5ee3 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sun, 7 Dec 2025 12:14:59 +0100 Subject: [PATCH 07/49] Add missing settings to recoverLostReplica When adding a new replica to an existing DatabaseReplicated cluster, the recoverLostReplica() function reads table metadata from ZooKeeper and recreates tables. However, the metadata stored in ZooKeeper contains only the CREATE TABLE statement and table settings, not the global settings that were active during original table creation. If a table was created with the DEFLATE_QPL codec (which requires the enable_deflate_qpl_codec global setting), a new replica would fail to create the table during recovery because this setting is not enabled. This fix explicitly enables enable_deflate_qpl_codec in the recovery query context, ensuring that tables using this codec can be successfully recreated on new replicas. Co-authored-by: Kevin Michel --- src/Databases/DatabaseReplicated.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 5bd167ae0ea5..0214c4bf9408 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1404,6 +1404,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setSetting("database_replicated_allow_explicit_uuid", 3); query_context->setSetting("database_replicated_allow_replicated_engine_arguments", 3); + query_context->setSetting("enable_deflate_qpl_codec", 1); /// We apply the flatten_nested setting after writing the CREATE query to the DDL log, /// but before writing metadata to ZooKeeper. So we have to apply the setting on secondary replicas, but not in recovery mode. From 392b9c7b1d6166d6b8a006a7603243518c02a92e Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sun, 7 Dec 2025 13:55:45 +0100 Subject: [PATCH 08/49] Fix unbounded replication queue growth If a replica is struggling to execute all tasks in its replication queue, the queue can grow to a huge size and becomes even slower to manage when it's larger (it's an std::list). This queue is also present in ZooKeeper, for each replica. There is also a shared queue, which is called the log. It's used to communicate changes between replicas. One replica will push an item on the log as part of executing a query, then all other replicas will read that log item and repush it in their own queue. Each replica monitors the log size and which replicas are deemed lost when they are too late to consume log items. However, the replicas do not monitor the size of the queue of the other replicas. If a replica is pushing lots of items in the log and another replica is able to copy them to their own queue, but too slow to consume the queue items, then we have a problem. At some point, if the imbalance persists, ZooKeeper will crash because it won't have enough memory to hold all the queues. It will be very hard to recover: ZooKeeper itself often corrupts its state when OOMing. Manually cleaning hundreds of thousands of items in the queues to know which ones can be removed and which ones are necessary to keep the replicated data in a consistent state is almost impossible. To fix that, we reuse the infrastructure that delays insert queries when there are too many parts. The same logic is applied when any replica queue size is too large for the table, or the grand total of all replicas' queue sizes, over all tables, is too large. We also do the same to fail queries if either counter reaches a second threshold, exactly like it's done with the parts count. In each replica, a thread is updating a local copy of the maximum queue size in all replicas. A map is added to the context to keep track of these per-storage maxima. These values are then used to gate the inserts. We do it like that to avoid adding many ZooKeeper queries in the hot path, and we don't need a very accurate queue size to get the feedback loop we need. Changes: - Add ReplicatedMergeTreeQueueSizeThread to periodically monitor queue sizes - Add settings: queue_size_to_delay_insert, queue_size_to_throw_insert, queues_total_size_to_delay_insert, queues_total_size_to_throw_insert - Add metric: ReplicatedQueuesTotalSize to track total queue size across all tables - Extend delayInsertOrThrowIfNeeded() to check queue sizes in addition to parts count - Add Context methods to track per-storage maximum queue sizes - Integrate queue size monitoring into StorageReplicatedMergeTree startup and shutdown lifecycle Co-authored-by: Kevin Michel --- docs/en/operations/system-tables/metrics.md | 4 + src/Common/CurrentMetrics.cpp | 1 + src/Core/Settings.cpp | 15 +++ src/Interpreters/Context.cpp | 30 ++++++ src/Interpreters/Context.h | 4 + src/Storages/MergeTree/MergeTreeData.cpp | 100 ++++++++++++++++-- src/Storages/MergeTree/MergeTreeData.h | 8 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 4 + .../ReplicatedMergeTreeQueueSizeThread.cpp | 37 +++++++ .../ReplicatedMergeTreeQueueSizeThread.h | 38 +++++++ .../MergeTree/ReplicatedMergeTreeSink.cpp | 5 +- src/Storages/StorageReplicatedMergeTree.cpp | 33 ++++++ src/Storages/StorageReplicatedMergeTree.h | 8 ++ 13 files changed, 275 insertions(+), 12 deletions(-) create mode 100644 src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.cpp create mode 100644 src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.h diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 8463ce7d9822..db908c175fa6 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -628,6 +628,10 @@ Number of data parts checking for consistency Number of data parts being fetched from replica +### ReplicatedQueuesTotalSize + +Total number of items contained in all nodes' replicated queues. + ### ReplicatedSend {#replicatedsend} Number of data parts being sent to replicas diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 1b4071cbb5dc..c7130ea42d55 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -14,6 +14,7 @@ M(ReplicatedFetch, "Number of data parts being fetched from replica") \ M(ReplicatedSend, "Number of data parts being sent to replicas") \ M(ReplicatedChecks, "Number of data parts checking for consistency") \ + M(ReplicatedQueuesTotalSize, "Total number of items contained in all nodes' replicated queues.") \ M(BackgroundMergesAndMutationsPoolTask, "Number of active merges and mutations in an associated background pool") \ M(BackgroundMergesAndMutationsPoolSize, "Limit on number of active merges and mutations in an associated background pool") \ M(BackgroundFetchesPoolTask, "Number of active fetches in an associated background pool") \ diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 903d984dee49..5698a752f795 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -2131,6 +2131,21 @@ If the destination table contains at least that many active parts in a single pa )", 0) \ DECLARE(UInt64, parts_to_throw_insert, 0, R"( If more than this number active parts in a single partition of the destination table, throw 'Too many parts ...' exception. +)", 0) \ +DECLARE(UInt64, queue_size_to_delay_insert, 50000, R"( +If the destination table replication queue is at least that large on any replica, artificially slow down insert into table. +)", 0) \ +DECLARE(UInt64, queue_size_to_throw_insert, 100000, R"( +If the destination table replication queue is at least that large on any replica, throw 'Too large replication queue ...' exception. +)", 0) \ +DECLARE(UInt64, queues_total_size_to_delay_insert, 100000, R"( +If the sum, for all tables, of the largest replication queue's size over all replicas is larger than this value, artificially slow down insert into table. +)", 0) \ +DECLARE(UInt64, queues_total_size_to_throw_insert, 200000, R"( +If the sum, for all tables, of the largest replication queue's size over all replicas is larger than this value, throw 'Too large replication queue ...' exception. +)", 0) \ +DECLARE(Bool, queue_size_monitor, true, R"( +If setting is enabled, monitor the replication queue on other replicas. )", 0) \ DECLARE(UInt64, number_of_mutations_to_delay, 0, R"( If the mutated table contains at least that many unfinished mutations, artificially slow down mutations of table. 0 - disabled diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e1a1e3b677c1..7f431b59c71b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -236,6 +237,7 @@ namespace CurrentMetrics extern const Metric IcebergCatalogThreads; extern const Metric IcebergCatalogThreadsActive; extern const Metric IcebergCatalogThreadsScheduled; + extern const Metric ReplicatedQueuesTotalSize; extern const Metric IndexMarkCacheBytes; extern const Metric IndexMarkCacheFiles; extern const Metric MarkCacheBytes; @@ -616,6 +618,10 @@ struct ContextSharedPart : boost::noncopyable std::optional storage_azure_settings TSA_GUARDED_BY(mutex); /// Settings of AzureBlobStorage std::unordered_map warnings TSA_GUARDED_BY(mutex); /// Store warning messages about server. + mutable std::mutex replicated_storage_queues_size_mutex; /// Guards the map below from concurrent access + std::unordered_map replicated_storage_queue_sizes{}; /// Local per-storage replicated queue size + std::atomic_size_t replicated_storage_queues_total_size = 0; /// Sum total of the map values above + /// Background executors for *MergeTree tables /// Has background executors for MergeTree tables been initialized? mutable ContextSharedMutex background_executors_mutex; @@ -5009,6 +5015,30 @@ size_t Context::getClustersVersion() const return shared->clusters_version; } +void Context::setStorageReplicatedQueuesSize(const UUID & storage_uuid, const size_t & replicated_queue_size) +{ + std::lock_guard lock(shared->replicated_storage_queues_size_mutex); + auto [elem_iter, inserted] = shared->replicated_storage_queue_sizes.insert({storage_uuid, replicated_queue_size}); + if (inserted) { + shared->replicated_storage_queues_total_size += replicated_queue_size; + } else { + shared->replicated_storage_queues_total_size += replicated_queue_size - elem_iter->second; + elem_iter->second = replicated_queue_size; + } + CurrentMetrics::set(CurrentMetrics::ReplicatedQueuesTotalSize, shared->replicated_storage_queues_total_size); +} + +void Context::clearStorageReplicatedQueueSize(const UUID & storage_uuid) +{ + Context::setStorageReplicatedQueuesSize(storage_uuid, 0); +} + +UInt64 Context::getReplicatedQueuesTotalSize() const +{ + return shared->replicated_storage_queues_total_size; +} + + void Context::setCluster(const String & cluster_name, const std::shared_ptr & cluster) { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 626705099e22..aaa6a2bc9a64 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1397,6 +1397,10 @@ class Context: public ContextData, public std::enable_shared_from_this void setConfigReloaderInterval(size_t value_ms); size_t getConfigReloaderInterval() const; + void setStorageReplicatedQueuesSize(const UUID & storage_uuid, const size_t & replicated_queue_size); + void clearStorageReplicatedQueueSize(const UUID & storage_uuid); + UInt64 getReplicatedQueuesTotalSize() const; + /// Lets you select the compression codec according to the conditions described in the configuration file. std::shared_ptr chooseCompressionCodec(size_t part_size, double part_size_ratio) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d1fea772e267..3e88b576175a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -194,6 +194,10 @@ namespace Setting extern const SettingsBool parallel_replicas_for_non_replicated_merge_tree; extern const SettingsUInt64 parts_to_delay_insert; extern const SettingsUInt64 parts_to_throw_insert; + extern const SettingsUInt64 queue_size_to_delay_insert; + extern const SettingsUInt64 queue_size_to_throw_insert; + extern const SettingsUInt64 queues_total_size_to_delay_insert; + extern const SettingsUInt64 queues_total_size_to_throw_insert; extern const SettingsBool enable_shared_storage_snapshot_in_query; extern const SettingsUInt64 merge_tree_storage_snapshot_sleep_ms; extern const SettingsUInt64 min_insert_block_size_rows; @@ -262,6 +266,10 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool enable_block_number_column; extern const MergeTreeSettingsBool enable_block_offset_column; extern const MergeTreeSettingsBool columns_and_secondary_indices_sizes_lazy_calculation; + extern const MergeTreeSettingsUInt64 queue_size_to_delay_insert; + extern const MergeTreeSettingsUInt64 queue_size_to_throw_insert; + extern const MergeTreeSettingsUInt64 queues_total_size_to_delay_insert; + extern const MergeTreeSettingsUInt64 queues_total_size_to_throw_insert; extern const MergeTreeSettingsSeconds refresh_parts_interval; extern const MergeTreeSettingsBool remove_unused_patch_parts; extern const MergeTreeSettingsSearchOrphanedPartsDisks search_orphaned_parts_disks; @@ -5397,7 +5405,11 @@ std::optional MergeTreeData::getMinPartDataVersion() const } -void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context, bool allow_throw) const +void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, + const ContextPtr & query_context, + bool allow_throw, + std::optional max_replicas_queue_size, + std::optional max_replicas_queues_total_size) const { const auto settings = getSettings(); const auto & query_settings = query_context->getSettingsRef(); @@ -5460,12 +5472,56 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex active_parts_over_threshold = parts_count_in_partition - active_parts_to_delay_insert + 1; } + size_t queue_size = max_replicas_queue_size.value_or(0); + auto queue_size_to_delay_insert = query_settings[Setting::queue_size_to_delay_insert].changed ? query_settings[Setting::queue_size_to_delay_insert] : (*settings)[MergeTreeSetting::queue_size_to_delay_insert]; + auto queue_size_to_throw_insert = query_settings[Setting::queue_size_to_throw_insert].changed ? query_settings[Setting::queue_size_to_throw_insert] : (*settings)[MergeTreeSetting::queue_size_to_throw_insert]; + + size_t queues_total_size = max_replicas_queues_total_size.value_or(0); + auto queues_total_size_to_delay_insert = query_settings[Setting::queues_total_size_to_delay_insert].changed + ? query_settings[Setting::queues_total_size_to_delay_insert] + : (*settings)[MergeTreeSetting::queues_total_size_to_delay_insert]; + auto queues_total_size_to_throw_insert = query_settings[Setting::queues_total_size_to_throw_insert].changed + ? query_settings[Setting::queues_total_size_to_throw_insert] + : (*settings)[MergeTreeSetting::queues_total_size_to_throw_insert]; + + if (queue_size >= queue_size_to_throw_insert) + { + ProfileEvents::increment(ProfileEvents::RejectedInserts); + throw Exception( + ErrorCodes::LIMIT_EXCEEDED, + "Too large replication queue ({}). One of the replicas is too slow for inserts", + queue_size); + } + if (queues_total_size >= queues_total_size_to_throw_insert) + { + ProfileEvents::increment(ProfileEvents::RejectedInserts); + throw Exception( + ErrorCodes::LIMIT_EXCEEDED, + "Too many item in replication queues ({}). One or multiple replicas are too slow for inserts", + queues_total_size); + } + + size_t queue_size_over_threshold = 0; + if (queue_size >= queue_size_to_delay_insert) + { + // if queue_size == queue_size_to_delay_insert -> we're 1 part over threshold + queue_size_over_threshold = queue_size - queue_size_to_delay_insert + 1; + } + + size_t queues_total_size_over_threshold = 0; + if (queues_total_size >= queues_total_size_to_delay_insert) + { + queues_total_size_over_threshold = queues_total_size - queues_total_size_to_delay_insert + 1; + } + /// no need for delay - if (!active_parts_over_threshold && !outdated_parts_over_threshold) + if (!active_parts_over_threshold && !outdated_parts_over_threshold && !queue_size_over_threshold && !queues_total_size_over_threshold) return; - UInt64 delay_milliseconds = 0; - { + UInt64 delay_milliseconds_parts = 0; + const UInt64 min_delay_milliseconds = (*settings)[MergeTreeSetting::min_delay_to_insert_ms]; + const UInt64 max_delay_milliseconds = ((*settings)[MergeTreeSetting::max_delay_to_insert] > 0 ? (*settings)[MergeTreeSetting::max_delay_to_insert] * 1000 : 1000); + if (active_parts_over_threshold || outdated_parts_over_threshold) { size_t parts_over_threshold = 0; size_t allowed_parts_over_threshold = 1; const bool use_active_parts_threshold = (active_parts_over_threshold >= outdated_parts_over_threshold); @@ -5482,26 +5538,50 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex allowed_parts_over_threshold = (*settings)[MergeTreeSetting::inactive_parts_to_throw_insert] - (*settings)[MergeTreeSetting::inactive_parts_to_delay_insert]; } - const UInt64 max_delay_milliseconds = ((*settings)[MergeTreeSetting::max_delay_to_insert] > 0 ? (*settings)[MergeTreeSetting::max_delay_to_insert] * 1000 : 1000); if (allowed_parts_over_threshold == 0 || parts_over_threshold > allowed_parts_over_threshold) { - delay_milliseconds = max_delay_milliseconds; + delay_milliseconds_parts = max_delay_milliseconds; } else { double delay_factor = static_cast(parts_over_threshold) / allowed_parts_over_threshold; - const UInt64 min_delay_milliseconds = (*settings)[MergeTreeSetting::min_delay_to_insert_ms]; - delay_milliseconds = std::max(min_delay_milliseconds, static_cast(max_delay_milliseconds * delay_factor)); + delay_milliseconds_parts = std::max(min_delay_milliseconds, static_cast(max_delay_milliseconds * delay_factor)); } } + UInt64 delay_milliseconds_queue = 0; + if (queue_size_over_threshold) + { + UInt64 allowed_queue_size_over_threshold = std::max(1ul, queue_size_to_throw_insert - queue_size_to_delay_insert); + double delay_factor = static_cast(queue_size_over_threshold) / allowed_queue_size_over_threshold; + delay_milliseconds_queue = std::max(min_delay_milliseconds, static_cast(max_delay_milliseconds * delay_factor)); + } + + UInt64 delay_milliseconds_queues_total = 0; + if (queues_total_size_over_threshold) + { + UInt64 allowed_queues_total_size_over_threshold + = std::max(1ul, queues_total_size_to_throw_insert - queues_total_size_to_delay_insert); + double queues_total_delay_factor = static_cast(queues_total_size_over_threshold) / allowed_queues_total_size_over_threshold; + delay_milliseconds_queues_total + = std::max(min_delay_milliseconds, static_cast(max_delay_milliseconds * queues_total_delay_factor)); + } + + const UInt64 delay_milliseconds = std::max({delay_milliseconds_parts, delay_milliseconds_queue, delay_milliseconds_queues_total}); + ProfileEvents::increment(ProfileEvents::DelayedInserts); ProfileEvents::increment(ProfileEvents::DelayedInsertsMilliseconds, delay_milliseconds); CurrentMetrics::Increment metric_increment(CurrentMetrics::DelayedInserts); - LOG_INFO(log, "Delaying inserting block by {} ms. because there are {} parts and their average size is {}", - delay_milliseconds, parts_count_in_partition, ReadableSize(average_part_size)); + if (delay_milliseconds_parts > delay_milliseconds_queue && delay_milliseconds_parts > delay_milliseconds_queues_total) + LOG_INFO(log, "Delaying inserting block by {} ms. because there are {} parts and their average size is {}", + delay_milliseconds, parts_count_in_partition, ReadableSize(average_part_size)); + else if (delay_milliseconds_queue > delay_milliseconds_queues_total) + LOG_INFO(log, "Delaying inserting block by {} ms. because replication queue size is {}", delay_milliseconds, queue_size); + else + LOG_INFO( + log, "Delaying inserting block by {} ms. because replication queues total size is {}", delay_milliseconds, queues_total_size); if (until) until->tryWait(delay_milliseconds); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2cd69c086473..8c16ae71fe22 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -89,6 +89,7 @@ using PartitionIdToMaxBlock = std::unordered_map; namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int LIMIT_EXCEEDED; } struct DataPartsLock @@ -738,7 +739,12 @@ class MergeTreeData : public IStorage, public WithMutableContext /// If the table contains too many active parts, sleep for a while to give them time to merge. /// If until is non-null, wake up from the sleep earlier if the event happened. /// The decision to delay or throw is made according to settings 'parts_to_delay_insert' and 'parts_to_throw_insert'. - void delayInsertOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context, bool allow_throw) const; + void delayInsertOrThrowIfNeeded( + Poco::Event * until, + const ContextPtr & query_context, + bool allow_throw, + std::optional max_replicas_queue_size = {}, + std::optional max_replicas_queues_total_size = {}) const; /// If the table contains too many unfinished mutations, sleep for a while to give them time to execute. /// If until is non-null, wake up from the sleep earlier if the event happened. diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index c73ca49f808c..d6ac882a8a78 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -699,6 +699,10 @@ namespace ErrorCodes Possible values: - Any positive integer. )", 0) \ + DECLARE(UInt64, queue_size_to_delay_insert, 50000, R"(If replication queue is at least that large on any replica, artificially slow down insert into table.)", 0) \ + DECLARE(UInt64, queue_size_to_throw_insert, 100000, R"(If replication queue is at least that large on any replica, throw 'Too large replication queue ...' exception.)", 0) \ + DECLARE(UInt64, queues_total_size_to_delay_insert, 100000, R"(If the sum, for all tables, of the largest replication queue's size over all replicas is larger than this value, artificially slow down insert into table.)", 0) \ + DECLARE(UInt64, queues_total_size_to_throw_insert, 200000, R"(If the sum, for all tables, of the largest replication queue's size over all replicas is larger than this value, throw 'Too large replication queue ...' exception.)", 0) \ DECLARE(UInt64, max_avg_part_size_for_too_many_parts, 1ULL * 1024 * 1024 * 1024, R"( The 'too many parts' check according to 'parts_to_delay_insert' and 'parts_to_throw_insert' will be active only if the average part size (in the diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.cpp new file mode 100644 index 000000000000..f58e384ac03b --- /dev/null +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +ReplicatedMergeTreeQueueSizeThread::ReplicatedMergeTreeQueueSizeThread(StorageReplicatedMergeTree & storage_) + : storage(storage_) + , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeQueueSizeThread)") + , log(&Poco::Logger::get(log_name)) +{ + task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); }); +} + +void ReplicatedMergeTreeQueueSizeThread::run() +{ + try + { + iterate(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + task->scheduleAfter(1000); +} + +void ReplicatedMergeTreeQueueSizeThread::iterate() +{ + storage.updateMaxReplicasQueueSize(); +} + +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.h new file mode 100644 index 000000000000..1b5e638b645d --- /dev/null +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueueSizeThread.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class StorageReplicatedMergeTree; + +/** Update the max queue size of all replicas + */ +class ReplicatedMergeTreeQueueSizeThread +{ +public: + explicit ReplicatedMergeTreeQueueSizeThread(StorageReplicatedMergeTree & storage_); + + void start() { task->activateAndSchedule(); } + + void wakeup() { task->schedule(); } + + void shutdown() { task->deactivate(); } + +private: + StorageReplicatedMergeTree & storage; + String log_name; + Poco::Logger * log; + BackgroundSchedulePool::TaskHolder task; + + void run(); + void iterate(); +}; + +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index b55f2e3c8aee..56bd38edf7c7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1192,7 +1192,10 @@ void ReplicatedMergeTreeSinkImpl::onStart() { /// It's only allowed to throw "too many parts" before write, /// because interrupting long-running INSERT query in the middle is not convenient for users. - storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, true); + auto max_replica_queue_size = storage.max_replicas_queue_size.load(std::memory_order_relaxed); + auto replicated_queues_total_size = context->getReplicatedQueuesTotalSize(); + storage.delayInsertOrThrowIfNeeded( + &storage.partial_shutdown_event, context, true, max_replica_queue_size, replicated_queues_total_size); } template diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3a4374a6c9dc..7efc6a4fcc11 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -183,6 +183,7 @@ namespace Setting extern const SettingsInt64 replication_wait_for_inactive_replica_timeout; extern const SettingsUInt64 select_sequential_consistency; extern const SettingsBool update_sequential_consistency; + extern const SettingsBool queue_size_monitor; } namespace MergeTreeSetting @@ -416,6 +417,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , fetcher(*this) , cleanup_thread(*this) , async_block_ids_cache(*this) + , queue_size_thread(*this) , part_check_thread(*this) , restarting_thread(*this) , part_moves_between_shards_orchestrator(*this) @@ -1410,6 +1412,12 @@ void StorageReplicatedMergeTree::drop() dropZookeeperZeroCopyLockPaths(zookeeper, zero_copy_locks_paths, log.load()); } } + /// Clear the replicated queue size count for the table. + /// It will be recomputed and updated to the true new count during the queue monitor thread's next iteration. + /// It's better to have a conservative total during two monitor iterations rather than pay the network cost + /// of iterating over the znodes' tree at this point. + auto storage_uuid = getStorageID().uuid; + getContext()->clearStorageReplicatedQueueSize(storage_uuid); } @@ -4038,6 +4046,26 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne return true; } +void StorageReplicatedMergeTree::updateMaxReplicasQueueSize() { + size_t max_replicas_queue_size_local = 0; + { + auto zookeeper = getZooKeeper(); + Coordination::Stat stat; + Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas", &stat); + for (const String & replica : replicas) + { + Coordination::Stat replica_stat; + zookeeper->get(zookeeper_path + "/replicas/" + replica + "/queue", &replica_stat); + size_t queue_size = replica_stat.numChildren; + max_replicas_queue_size_local = std::max(max_replicas_queue_size_local, queue_size); + } + } + LOG_TRACE(log, "Max replica queue size is {}", max_replicas_queue_size_local); + max_replicas_queue_size.store(max_replicas_queue_size_local); + auto storage_uuid = getStorageID().uuid; + getContext()->setStorageReplicatedQueuesSize(storage_uuid, max_replicas_queue_size_local); +} + bool StorageReplicatedMergeTree::canExecuteFetch(const ReplicatedMergeTreeLogEntry & entry, String & disable_reason) const { @@ -5634,6 +5662,10 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread, const ZooK restarting_thread.start(true); }); + if (getContext()->getSettingsRef()[Setting::queue_size_monitor]) + queue_size_thread.start(); + + startBackgroundMovesIfNeeded(); part_moves_between_shards_orchestrator.start(); @@ -5703,6 +5735,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() LOG_TRACE(log, "The attach thread is shutdown"); } + queue_size_thread.shutdown(); restarting_thread.shutdown(/* part_of_full_shutdown */true); /// Explicitly set the event, because the restarting thread will not set it again startup_event.set(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 487c3a3f44c0..6a76f234b8ae 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -264,6 +265,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// Schedules job to execute in background pool (merge, mutate, drop range and so on) bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; + // Update max_replicas_queue_size from ZooKeeper + void updateMaxReplicasQueueSize(); + /// Checks that fetches are not disabled with action blocker and pool for fetches /// is not overloaded bool canExecuteFetch(const ReplicatedMergeTreeLogEntry & entry, String & disable_reason) const; @@ -462,6 +466,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData ReplicatedMergeTreeQueue queue; std::atomic last_queue_update_start_time{0}; std::atomic last_queue_update_finish_time{0}; + std::atomic max_replicas_queue_size{0}; mutable std::mutex last_queue_update_exception_lock; String last_queue_update_exception; @@ -515,6 +520,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData AsyncBlockIDsCache async_block_ids_cache; + /// A thread that updates the maximum queue size of all replicas. + ReplicatedMergeTreeQueueSizeThread queue_size_thread; + /// A thread that checks the data of the parts, as well as the queue of the parts to be checked. ReplicatedMergeTreePartCheckThread part_check_thread; From 18f662393bb54e2edc3154a465ab05a9dc392fc6 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sun, 7 Dec 2025 14:20:46 +0100 Subject: [PATCH 09/49] Replicate ALTER TABLE MOVE PARTITION queries through DatabaseReplicated Previously, MOVE PARTITION commands were not replicated through the DatabaseReplicated DDL log, causing data inconsistency when moving partitions between tables across replicas in a DatabaseReplicated cluster. The issue occurred because shouldReplicateQuery() only checked for AlterCommand (metadata changes) but not for PartitionCommand::MOVE_PARTITION. Other partition commands like DROP PARTITION and ATTACH PARTITION are handled by ReplicatedMergeTree's own replication mechanism, but MOVE PARTITION TO TABLE requires database-level coordination to ensure both source and destination tables are updated consistently across all replicas. Changes: - Add PartitionCommands.h include - Extend shouldReplicateQuery() to detect MOVE_PARTITION commands - Return true for MOVE_PARTITION to enable database-level replication This ensures that when a partition is moved between tables, all replicas execute the operation atomically, maintaining data consistency across the cluster. Co-authored-by: Joe Lynch --- src/Databases/DatabaseReplicated.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 0214c4bf9408..b2688388adcf 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -55,6 +55,7 @@ #include #include #include +#include "Storages/PartitionCommands.h" namespace DB @@ -2282,11 +2283,18 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context, try { /// Metadata alter should go through database - for (const auto & child : alter->command_list->children) - if (AlterCommand::parse(child->as())) + for (const auto & child : alter->command_list->children) { + auto * const child_command = child->as(); + if (AlterCommand::parse(child_command)) return true; + else { + auto const partition_command = PartitionCommand::parse(child_command); + if (partition_command && partition_command->type == PartitionCommand::MOVE_PARTITION) + return true; + } + } - /// It's ALTER PARTITION or mutation, doesn't involve database + /// It's a non-moving ALTER PARTITION or mutation, doesn't involve database return false; } catch (...) From 4f53d4e5ac3270d4a714ec62057ff43c4020c09a Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 8 Dec 2025 14:02:38 +0100 Subject: [PATCH 10/49] Change default logs_to_keep from 1000 to 300 for DatabaseReplicated Reduce the default value of setting from 1000 to 300 for DatabaseReplicated databases. This reduces ZooKeeper resource consumption by ~70% while maintaining a 6x safety margin over max_replication_lag_to_enqueue (50). Context: Previously, we implemented a server-level setting (replicated_database_logs_to_keep) to centralize control of this value. However, after analysis, we determined that: 1. Customers do not have ALTER_DATABASE_SETTINGS permission, so they cannot modify database settings via ALTER DATABASE MODIFY SETTING 2. The simpler approach of changing the database-level default is sufficient 3. No additional readonly checks are needed since access control already prevents customer modifications This change affects only newly created databases. Existing databases retain their current logs_to_keep value stored in ZooKeeper. The default value of 300 provides adequate recovery buffer while significantly reducing ZooKeeper memory usage in multi-database managed provider environments. Co-authored-by: Khatskevich --- src/Databases/DatabaseReplicatedSettings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicatedSettings.cpp b/src/Databases/DatabaseReplicatedSettings.cpp index 14eca24b3569..1de07d34ad39 100644 --- a/src/Databases/DatabaseReplicatedSettings.cpp +++ b/src/Databases/DatabaseReplicatedSettings.cpp @@ -22,7 +22,7 @@ extern const int UNKNOWN_SETTING; DECLARE(Bool, check_consistency, true, "Check consistency of local metadata and metadata in Keeper, do replica recovery on inconsistency", 0) \ DECLARE(UInt64, max_retries_before_automatic_recovery, 10, "Max number of attempts to execute a queue entry before marking replica as lost recovering it from snapshot (0 means infinite)", 0) \ DECLARE(Bool, allow_skipping_old_temporary_tables_ddls_of_refreshable_materialized_views, false, "If enabled, when processing DDLs in Replicated databases, it skips creating and exchanging DDLs of the temporary tables of refreshable materialized views if possible", 0) \ - DECLARE(NonZeroUInt64, logs_to_keep, 1000, "Default number of logs to keep in ZooKeeper for Replicated database.", 0) \ + DECLARE(NonZeroUInt64, logs_to_keep, 300, "Default number of logs to keep in ZooKeeper for Replicated database.", 0) \ DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS) IMPLEMENT_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS) From b8d335863b9421518af08aa5ed6b70847a239589 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 9 Dec 2025 13:36:21 +0100 Subject: [PATCH 11/49] Restrict SHOW CREATE DATABASE access Main service users are allowed the SHOW DATABASES access because it is necessary for their operation. It is implicitly granted by ClickHouse when giving access to a database. However, we do not want to give them access to SHOW CREATE DATABASE. This query shows the entire create statement, unredacted. This is actually a useful feature for superusers, but can leak credentials to other users. Also only show the create query in system.tables to users that were able to create that table. Co-authored-by: Kevin Michel --- src/Interpreters/InterpreterShowCreateQuery.cpp | 10 ++++++++-- src/Storages/System/StorageSystemTables.cpp | 4 +++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 381834f09580..082388a0ceef 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -58,10 +59,14 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl() bool is_dictionary = static_cast(query_ptr->as()); - if (is_dictionary) + if (is_dictionary) { getContext()->checkAccess(AccessType::SHOW_DICTIONARIES, table_id); - else + } + + else { getContext()->checkAccess(AccessType::SHOW_COLUMNS, table_id); + getContext()->checkAccess(AccessType::CREATE_TABLE, table_id); + } create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, getContext()); @@ -85,6 +90,7 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl() throw Exception(ErrorCodes::SYNTAX_ERROR, "Temporary databases are not possible."); show_query->setDatabase(getContext()->resolveDatabase(show_query->getDatabase())); getContext()->checkAccess(AccessType::SHOW_DATABASES, show_query->getDatabase()); + getContext()->checkAccess(AccessType::CREATE_DATABASE, show_query->getDatabase()); create_query = DatabaseCatalog::instance().getDatabase(show_query->getDatabase())->getCreateDatabaseQuery(); } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index ff1d570a8024..6bc8e8584fbe 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -557,7 +558,8 @@ class TablesBlockSource : public ISource if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) { - ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); + auto const can_create_table = access->isGranted(AccessType::CREATE_TABLE, database_name, table_name); + ASTPtr ast = can_create_table ? database->tryGetCreateTableQuery(table_name, context): nullptr; auto * ast_create = ast ? ast->as() : nullptr; if (ast_create && !context->getSettingsRef()[Setting::show_table_uuid_in_table_create_query_if_not_nil]) From 83f8c091014524270a53de0bca2a5e9a47a5c2c9 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 9 Dec 2025 15:45:05 +0100 Subject: [PATCH 12/49] Allow custom CA certificate path for S3 connections This change enables S3 object storage to use custom CA certificates for HTTPS connections, which is useful for testing environments or private deployments where certificates are not signed by public CAs. The implementation adds a configuration option that can be specified in S3 disk settings. When provided, this path is used to create a custom SSL context for all HTTPS connections to that S3 endpoint. Key changes: - Added field to S3Settings struct with config loading, serialization, and deserialization support - Extended PocoHTTPClientConfiguration to store and pass ca_path - Updated ClientFactory::createClientConfiguration to accept ca_path - Modified PocoHTTPClient to create SSL context from ca_path when provided - Extended HTTP layer (HTTPCommon, HTTPConnectionPool) to accept and use custom SSL contexts - Updated all makeHTTPSession call sites to match new signature - Preserved ca_path through credentials provider chain Backwards compatibility: - ca_path is optional and defaults to std::nullopt - Serialization includes backwards compatibility handling for old data formats without ca_path - All existing call sites continue to work with default (empty) context parameter Co-authored-by: Kevin Michel --- src/Access/HTTPAuthClient.h | 5 ++- src/Backups/BackupIO_S3.cpp | 1 + src/Common/HTTPConnectionPool.cpp | 35 +++++++++++------ src/Common/HTTPConnectionPool.h | 3 +- .../RemoteProxyConfigurationResolver.cpp | 5 ++- src/Coordination/KeeperSnapshotManagerS3.cpp | 1 + src/Databases/DataLake/GlueCatalog.cpp | 1 + src/Disks/ObjectStorages/S3/diskSettings.cpp | 1 + src/IO/AzureBlobStorage/PocoHTTPClient.cpp | 6 ++- src/IO/HTTPCommon.cpp | 9 ++++- src/IO/HTTPCommon.h | 4 +- src/IO/ReadWriteBufferFromHTTP.cpp | 5 ++- src/IO/S3/Client.cpp | 2 + src/IO/S3/Client.h | 1 + src/IO/S3/Credentials.cpp | 3 ++ src/IO/S3/PocoHTTPClient.cpp | 19 +++++++++- src/IO/S3/PocoHTTPClient.h | 3 ++ src/IO/S3/tests/gtest_aws_s3_client.cpp | 2 + src/IO/S3Settings.cpp | 38 ++++++++++++++++++- src/IO/S3Settings.h | 1 + src/IO/WriteBufferFromHTTP.cpp | 5 ++- src/IO/tests/gtest_readbuffer_s3.cpp | 1 + src/IO/tests/gtest_writebuffer_s3.cpp | 1 + .../Formats/Impl/AvroRowInputFormat.cpp | 6 ++- 24 files changed, 133 insertions(+), 25 deletions(-) diff --git a/src/Access/HTTPAuthClient.h b/src/Access/HTTPAuthClient.h index cec9b79f2559..f8ceabd10ecf 100644 --- a/src/Access/HTTPAuthClient.h +++ b/src/Access/HTTPAuthClient.h @@ -39,7 +39,10 @@ class HTTPAuthClient Result authenticateRequest(Poco::Net::HTTPRequest & request) const { - auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, timeouts); + // Note: makeHTTPSession signature was updated to support custom CA certificates for S3. + // This call site was already using makeHTTPSession, but needed to be updated to match the new signature. + // For HTTP authentication requests, we pass an empty context (default) since we don't need custom CA certificates. + auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, timeouts, {}, nullptr, {}); Poco::Net::HTTPResponse response; auto milliseconds_to_wait = retry_initial_backoff_ms; diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 51cf22a12eee..6d88f4294352 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -146,6 +146,7 @@ class S3BackupClientCreator local_settings[Setting::s3_slow_all_threads_after_network_error], local_settings[Setting::backup_slow_all_threads_after_retryable_s3_error], local_settings[Setting::enable_s3_requests_logging], + /* ca_path */ std::optional(), /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, request_settings.get_request_throttler, diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index a6db50c05455..7ad620403825 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -650,12 +651,13 @@ class EndpointConnectionPool : public std::enable_shared_from_this context_) : host(std::move(host_)) , port(port_) , https(https_) , proxy_configuration(std::move(proxy_configuration_)) , group(group_) + , context(context_) { } @@ -671,7 +673,11 @@ class EndpointConnectionPool : public std::enable_shared_from_thisgetWeakFromThis(), group, getMetrics(), host, port); + ConnectionPtr connection; + if constexpr (std::is_same_v) + connection = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port, context); + else + connection = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port); connection->setKeepAlive(true); setTimeouts(*connection, timeouts); @@ -716,7 +722,11 @@ class EndpointConnectionPool : public std::enable_shared_from_thisgetWeakFromThis(), group, getMetrics(), host, port); + ConnectionPtr connection_to_store; + if constexpr (std::is_same_v) + connection_to_store = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port, context); + else + connection_to_store = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port); connection_to_store->assign(connection); { @@ -735,6 +745,7 @@ class EndpointConnectionPool : public std::enable_shared_from_this context; std::mutex mutex; ConnectionsMinHeap stored_connections TSA_GUARDED_BY(mutex); @@ -794,13 +805,13 @@ struct Hasher }; static IExtendedPool::Ptr -createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port, bool secure, ProxyConfiguration proxy_configuration) +createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port, bool secure, ProxyConfiguration proxy_configuration, Poco::AutoPtr context = {}) { if (secure) { #if USE_SSL return EndpointConnectionPool::create( - group, std::move(host), port, secure, std::move(proxy_configuration)); + group, std::move(host), port, secure, std::move(proxy_configuration), context); #else throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS support is disabled, because ClickHouse was built without SSL library"); @@ -809,7 +820,7 @@ createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port, else { return EndpointConnectionPool::create( - group, std::move(host), port, secure, std::move(proxy_configuration)); + group, std::move(host), port, secure, std::move(proxy_configuration), context); } } @@ -832,7 +843,7 @@ class HTTPConnectionPools::Impl Poco::Timestamp last_wipe_time TSA_GUARDED_BY(mutex); public: - IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration, Poco::AutoPtr context) { Poco::Timestamp now; @@ -844,7 +855,7 @@ class HTTPConnectionPools::Impl last_wipe_time = now; } - return getPoolImpl(type, uri, proxy_configuration); + return getPoolImpl(type, uri, proxy_configuration, context); } void setLimits(HTTPConnectionPools::Limits disk, HTTPConnectionPools::Limits storage, HTTPConnectionPools::Limits http) @@ -874,7 +885,7 @@ class HTTPConnectionPools::Impl } } - IExtendedPool::Ptr getPoolImpl(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) + IExtendedPool::Ptr getPoolImpl(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration, Poco::AutoPtr context) TSA_REQUIRES(mutex) { auto [host, port, secure] = getHostPortSecure(uri, proxy_configuration); @@ -884,7 +895,7 @@ class HTTPConnectionPools::Impl if (it != endpoints_pool.end()) return it->second; - it = endpoints_pool.emplace(key, createConnectionPool(getGroup(type), std::move(host), port, secure, proxy_configuration)).first; + it = endpoints_pool.emplace(key, createConnectionPool(getGroup(type), std::move(host), port, secure, proxy_configuration, context)).first; return it->second; } @@ -954,8 +965,8 @@ void HTTPConnectionPools::dropCache() } IHTTPConnectionPoolForEndpoint::Ptr -HTTPConnectionPools::getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration) +HTTPConnectionPools::getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration, Poco::AutoPtr context) { - return impl->getPool(type, uri, proxy_configuration); + return impl->getPool(type, uri, proxy_configuration, context); } } diff --git a/src/Common/HTTPConnectionPool.h b/src/Common/HTTPConnectionPool.h index db0e4757a586..0b7deadcd1db 100644 --- a/src/Common/HTTPConnectionPool.h +++ b/src/Common/HTTPConnectionPool.h @@ -9,6 +9,7 @@ #include +#include #include #include @@ -83,7 +84,7 @@ class HTTPConnectionPools void setLimits(Limits disk, Limits storage, Limits http); void dropCache(); - IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration); + IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration, Poco::AutoPtr context = {}); private: class Impl; diff --git a/src/Common/RemoteProxyConfigurationResolver.cpp b/src/Common/RemoteProxyConfigurationResolver.cpp index d45a7a42c0b1..71a53cf267a5 100644 --- a/src/Common/RemoteProxyConfigurationResolver.cpp +++ b/src/Common/RemoteProxyConfigurationResolver.cpp @@ -19,7 +19,10 @@ namespace ErrorCodes std::string RemoteProxyHostFetcherImpl::fetch(const Poco::URI & endpoint, const ConnectionTimeouts & timeouts) { auto request = Poco::Net::HTTPRequest(Poco::Net::HTTPRequest::HTTP_GET, endpoint.getPath(), Poco::Net::HTTPRequest::HTTP_1_1); - auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, endpoint, timeouts); + // Note: makeHTTPSession signature was updated to support custom CA certificates for S3. + // This call site was already using makeHTTPSession, but needed to be updated to match the new signature. + // For proxy configuration resolution, we pass an empty context (default) since we don't need custom CA certificates. + auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, endpoint, timeouts, {}, nullptr, {}); session->sendRequest(request); diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 0973fdc045c9..74a0c1289ebd 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -122,6 +122,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo s3_slow_all_threads_after_network_error, s3_slow_all_threads_after_retryable_error, enable_s3_requests_logging, + /* ca_path = */ std::optional(), /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, diff --git a/src/Databases/DataLake/GlueCatalog.cpp b/src/Databases/DataLake/GlueCatalog.cpp index c0233f362ec7..6cb14ae965e7 100644 --- a/src/Databases/DataLake/GlueCatalog.cpp +++ b/src/Databases/DataLake/GlueCatalog.cpp @@ -121,6 +121,7 @@ GlueCatalog::GlueCatalog( s3_slow_all_threads_after_network_error, s3_slow_all_threads_after_retryable_error, enable_s3_requests_logging, + /* ca_path */ std::optional(), /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, /* get_request_throttler = */ nullptr, diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index e9c4fe2df80a..c486c8d3b5b2 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -122,6 +122,7 @@ getClient(const S3::URI & url, const S3Settings & settings, ContextPtr context, s3_slow_all_threads_after_network_error, /* s3_slow_all_threads_after_retryable_error = */ false, enable_s3_requests_logging, + settings.ca_path, for_disk_s3, opt_disk_name, request_settings.get_request_throttler, diff --git a/src/IO/AzureBlobStorage/PocoHTTPClient.cpp b/src/IO/AzureBlobStorage/PocoHTTPClient.cpp index 31e6f957e393..a31ec0e5680d 100644 --- a/src/IO/AzureBlobStorage/PocoHTTPClient.cpp +++ b/src/IO/AzureBlobStorage/PocoHTTPClient.cpp @@ -415,12 +415,16 @@ std::unique_ptr PocoAzureHTTPClient::makeRequest auto adaptive_timeouts = getTimeouts(method, first_attempt, true); auto group = for_disk_azure ? HTTPConnectionGroupType::DISK : HTTPConnectionGroupType::STORAGE; + // Note: makeHTTPSession signature was updated to support custom CA certificates for S3. + // This call site was already using makeHTTPSession, but needed to be updated to match the new signature. + // For Azure Blob Storage, we pass an empty context (default) since Azure doesn't use custom CA certificates. auto session = makeHTTPSession( group, uri, adaptive_timeouts, ProxyConfiguration{}, - &connect_time + &connect_time, + {} ); Stopwatch watch; diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index d25728de4582..caaf2e7adf4d 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -57,9 +57,14 @@ HTTPSessionPtr makeHTTPSession( const Poco::URI & uri, const ConnectionTimeouts & timeouts, const ProxyConfiguration & proxy_configuration, - UInt64 * connect_time) + UInt64 * connect_time, + Poco::AutoPtr context) { - auto connection_pool = HTTPConnectionPools::instance().getPool(group, uri, proxy_configuration); + if (!context) + { + context = Poco::Net::SSLManager::instance().defaultClientContext(); + } + auto connection_pool = HTTPConnectionPools::instance().getPool(group, uri, proxy_configuration, context); return connection_pool->getConnection(timeouts, connect_time); } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index abf249cddff2..d49118566afb 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -63,7 +64,8 @@ HTTPSessionPtr makeHTTPSession( const Poco::URI & uri, const ConnectionTimeouts & timeouts, const ProxyConfiguration & proxy_config = {}, - UInt64 * connect_time = nullptr + UInt64 * connect_time = nullptr, + Poco::AutoPtr context = {} ); bool isRedirect(Poco::Net::HTTPResponse::HTTPStatus status); diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index d1554262f7e9..07e411243e91 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -269,7 +269,10 @@ ReadWriteBufferFromHTTP::CallResult ReadWriteBufferFromHTTP::callImpl( Poco::Net::HTTPRequest request(method_, current_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); prepareRequest(request, range); - auto session = makeHTTPSession(connection_group, current_uri, timeouts, proxy_config); + // Note: makeHTTPSession signature was updated to support custom CA certificates for S3. + // This call site was already using makeHTTPSession, but needed to be updated to match the new signature. + // For generic HTTP read/write buffers, we pass an empty context (default) since we don't need custom CA certificates. + auto session = makeHTTPSession(connection_group, current_uri, timeouts, proxy_config, nullptr, {}); ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPRequestsSent); diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index c46d1456c417..31eb0789babc 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -1219,6 +1219,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT bool s3_slow_all_threads_after_network_error, bool s3_slow_all_threads_after_retryable_error, bool enable_s3_requests_logging, + const std::optional & ca_path, bool for_disk_s3, std::optional opt_disk_name, const ThrottlerPtr & get_request_throttler, @@ -1241,6 +1242,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT s3_slow_all_threads_after_network_error, s3_slow_all_threads_after_retryable_error, enable_s3_requests_logging, + ca_path, for_disk_s3, opt_disk_name, context->getGlobalContext()->getSettingsRef()[Setting::s3_use_adaptive_timeouts], diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 16457c7a511f..6d1b738840bf 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -358,6 +358,7 @@ class ClientFactory bool s3_slow_all_threads_after_network_error, bool s3_slow_all_threads_after_retryable_error, bool enable_s3_requests_logging, + const std::optional & ca_path, bool for_disk_s3, std::optional opt_disk_name, const ThrottlerPtr & get_request_throttler, diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index aed33b800c87..2986435d5451 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -741,6 +741,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.s3_slow_all_threads_after_network_error, configuration.s3_slow_all_threads_after_retryable_error, configuration.enable_s3_requests_logging, + configuration.ca_path, configuration.for_disk_s3, configuration.opt_disk_name, configuration.get_request_throttler, @@ -759,6 +760,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.s3_slow_all_threads_after_network_error, configuration.s3_slow_all_threads_after_retryable_error, configuration.enable_s3_requests_logging, + configuration.ca_path, configuration.for_disk_s3, configuration.opt_disk_name, configuration.get_request_throttler, @@ -812,6 +814,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.s3_slow_all_threads_after_network_error, configuration.s3_slow_all_threads_after_retryable_error, configuration.enable_s3_requests_logging, + configuration.ca_path, configuration.for_disk_s3, configuration.opt_disk_name, configuration.get_request_throttler, diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 7b36c5195a34..0a845eea7e4f 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include "config.h" @@ -103,6 +104,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( bool s3_slow_all_threads_after_network_error_, bool s3_slow_all_threads_after_retryable_error_, bool enable_s3_requests_logging_, + const std::optional & ca_path_, bool for_disk_s3_, std::optional opt_disk_name_, bool s3_use_adaptive_timeouts_, @@ -117,6 +119,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , s3_slow_all_threads_after_network_error(s3_slow_all_threads_after_network_error_) , s3_slow_all_threads_after_retryable_error(s3_slow_all_threads_after_retryable_error_) , enable_s3_requests_logging(enable_s3_requests_logging_) + , ca_path(ca_path_) , for_disk_s3(for_disk_s3_) , opt_disk_name(opt_disk_name_) , get_request_throttler(get_request_throttler_) @@ -189,6 +192,7 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config , http_max_field_value_size(client_configuration.http_max_field_value_size) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) , for_disk_s3(client_configuration.for_disk_s3) + , ca_path(client_configuration.ca_path) , get_request_throttler(client_configuration.get_request_throttler) , put_request_throttler(client_configuration.put_request_throttler) , extra_headers(client_configuration.extra_headers) @@ -515,6 +519,13 @@ void PocoHTTPClient::makeRequestInternalImpl( auto group = for_disk_s3 ? HTTPConnectionGroupType::DISK : HTTPConnectionGroupType::STORAGE; + Poco::AutoPtr context; + if (ca_path.has_value()) + { + context = Poco::AutoPtr( + new Poco::Net::Context(Poco::Net::Context::Usage::TLSV1_2_CLIENT_USE, ca_path.value(), Poco::Net::Context::VERIFY_RELAXED, 9, false)); + } + /// Reset times, as this may be a next session after a redirect connect_time = first_byte_time = 0; latency_recorded = false; @@ -523,7 +534,8 @@ void PocoHTTPClient::makeRequestInternalImpl( target_uri, getTimeouts(method, first_attempt, /*first_byte*/ true), proxy_configuration, - &connect_time); + &connect_time, + context); /// In case of error this address will be written to logs request.SetResolvedRemoteHost(session->getResolvedAddress()); @@ -800,7 +812,10 @@ PocoHTTPClientGCPOAuth::BearerToken PocoHTTPClientGCPOAuth::requestBearerToken() LOG_TEST(log, "Make request to: {}", url.toString()); auto group = for_disk_s3 ? HTTPConnectionGroupType::DISK : HTTPConnectionGroupType::STORAGE; - auto session = makeHTTPSession(group, url, timeouts); + // Note: This is for GCP OAuth token requests, not S3 data operations. This call site was already using + // makeHTTPSession, but needed to be updated to match the new signature. We pass an empty context (default) + // since GCP OAuth endpoints use standard CA certificates. S3 data operations use the context from ca_path. + auto session = makeHTTPSession(group, url, timeouts, {}, nullptr, {}); session->sendRequest(request); Poco::Net::HTTPResponse response; diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index db8f8ea415aa..6074f2125a62 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -59,6 +59,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration bool s3_slow_all_threads_after_network_error; bool s3_slow_all_threads_after_retryable_error; bool enable_s3_requests_logging; + std::optional ca_path; bool for_disk_s3; std::optional opt_disk_name; ThrottlerPtr get_request_throttler; @@ -93,6 +94,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration bool s3_slow_all_threads_after_network_error_, bool s3_slow_all_threads_after_retryable_error_, bool enable_s3_requests_logging_, + const std::optional & ca_path_, bool for_disk_s3_, std::optional opt_disk_name_, bool s3_use_adaptive_timeouts_, @@ -220,6 +222,7 @@ class PocoHTTPClient : public Aws::Http::HttpClient const UInt64 http_max_field_value_size = 128 * 1024; bool enable_s3_requests_logging = false; bool for_disk_s3 = false; + std::optional ca_path; /// Limits get request per second rate for GET, SELECT and all other requests, excluding throttled by put throttler /// (i.e. throttles GetObject, HeadObject) diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 9640b62dc7f2..1c7b2c17b834 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -140,6 +140,7 @@ void testServerSideEncryption( s3_slow_all_threads_after_network_error, s3_slow_all_threads_after_retryable_error, enable_s3_requests_logging, + /* ca_path = */ std::optional(), /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, @@ -416,6 +417,7 @@ TEST(IOTestAwsS3Client, AssumeRole) s3_slow_all_threads_after_network_error, s3_slow_all_threads_after_retryable_error, enable_s3_requests_logging, + /* ca_path = */ std::optional(), /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index 0fb323f42042..21abec620cb6 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include @@ -52,6 +54,15 @@ void S3Settings::loadFromConfigForObjectStorage( request_settings[S3RequestSetting::min_bytes_for_seek] = config.getUInt64(config_prefix + ".min_bytes_for_seek", S3::DEFAULT_MIN_BYTES_FOR_SEEK); request_settings[S3RequestSetting::list_object_keys_size] = config.getUInt64(config_prefix + ".list_object_keys_size", S3::DEFAULT_LIST_OBJECT_KEYS_SIZE); request_settings[S3RequestSetting::objects_chunk_size_to_delete] = config.getUInt(config_prefix + ".objects_chunk_size_to_delete", S3::DEFAULT_OBJECTS_CHUNK_SIZE_TO_DELETE); + + if (config.has(config_prefix + ".ca_path")) + { + ca_path = config.getString(config_prefix + ".ca_path"); + } + else + { + ca_path = std::nullopt; + } } @@ -59,6 +70,8 @@ void S3Settings::updateIfChanged(const S3Settings & settings) { auth_settings.updateIfChanged(settings.auth_settings); request_settings.updateIfChanged(settings.request_settings); + if (settings.ca_path.has_value()) + ca_path = settings.ca_path; } void S3SettingsByEndpoint::loadFromConfig( @@ -88,9 +101,12 @@ void S3SettingsByEndpoint::loadFromConfig( auto request_settings{default_request_settings}; request_settings.updateIfChanged(S3::S3RequestSettings(config, settings, key_path, "", settings[Setting::s3_validate_request_settings])); + S3Settings endpoint_settings{std::move(auth_settings), std::move(request_settings), std::nullopt}; + if (config.has(key_path + ".ca_path")) + endpoint_settings.ca_path = config.getString(key_path + ".ca_path"); s3_settings.emplace( config.getString(endpoint_path), - S3Settings{std::move(auth_settings), std::move(request_settings)}); + std::move(endpoint_settings)); } } } @@ -119,6 +135,9 @@ void S3Settings::serialize(WriteBuffer & os, ContextPtr context) const { auth_settings.serialize(os, context); request_settings.serialize(os, context); + writeBinary(ca_path.has_value(), os); + if (ca_path.has_value()) + writeStringBinary(ca_path.value(), os); } S3Settings S3Settings::deserialize(ReadBuffer & is, ContextPtr context) @@ -126,6 +145,23 @@ S3Settings S3Settings::deserialize(ReadBuffer & is, ContextPtr context) S3Settings result; result.auth_settings = S3::S3AuthSettings::deserialize(is, context); result.request_settings = S3::S3RequestSettings::deserialize(is, context); + // Backwards compatibility: ca_path was added later, so it might not be present in old data + bool has_ca_path = false; + try + { + readBinary(has_ca_path, is); + if (has_ca_path) + { + String path; + readStringBinary(path, is); + result.ca_path = path; + } + } + catch (...) + { + // Old format without ca_path - leave it as std::nullopt + result.ca_path = std::nullopt; + } return result; } diff --git a/src/IO/S3Settings.h b/src/IO/S3Settings.h index 76befc1fc52f..73a3c2365053 100644 --- a/src/IO/S3Settings.h +++ b/src/IO/S3Settings.h @@ -23,6 +23,7 @@ struct S3Settings { S3::S3AuthSettings auth_settings; S3::S3RequestSettings request_settings; + std::optional ca_path; /// Read all settings with from specified prefix in config. Expect settings name to start with "s3_". /// This method is useful when you need to get S3Settings for standalone S3 Client. diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index dc95d582fd8a..26119ad145db 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -26,7 +26,10 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( ProxyConfiguration proxy_configuration ) : WriteBufferFromOStream(buffer_size_) - , session{makeHTTPSession(connection_group, uri, timeouts, proxy_configuration)} + // Note: makeHTTPSession signature was updated to support custom CA certificates for S3. + // This call site was already using makeHTTPSession, but needed to be updated to match the new signature. + // For generic HTTP write buffers, we pass an empty context (default) since we don't need custom CA certificates. + , session{makeHTTPSession(connection_group, uri, timeouts, proxy_configuration, nullptr, {})} , request{method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} { if (uri.getPort()) diff --git a/src/IO/tests/gtest_readbuffer_s3.cpp b/src/IO/tests/gtest_readbuffer_s3.cpp index 4ee1d810de8d..616333cf8309 100644 --- a/src/IO/tests/gtest_readbuffer_s3.cpp +++ b/src/IO/tests/gtest_readbuffer_s3.cpp @@ -60,6 +60,7 @@ struct ClientFake : DB::S3::Client true, true, true, + /* ca_path = */ std::optional(), false, {}, {}, diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 444c19457dd8..74702949750b 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -248,6 +248,7 @@ struct Client : DB::S3::Client /* s3_slow_all_threads_after_network_error = */ true, /* s3_slow_all_threads_after_retryable_error = */ true, /* enable_s3_requests_logging = */ true, + /* ca_path = */ std::optional(), /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index f9a780b42007..1c67aa2f66d4 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1109,7 +1109,11 @@ class AvroConfluentRowInputFormat::SchemaRegistry http_basic_credentials.authenticate(request); } - auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, url, timeouts); + // Note: makeHTTPSession signature was updated to support custom CA certificates for S3. + // This call site was already using makeHTTPSession, but needed to be updated to match the new signature. + // For non-S3 HTTP requests (like Avro schema registry), we pass an empty context (default) + // since we don't need custom CA certificates. + auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, url, timeouts, {}, nullptr, {}); session->sendRequest(request); Poco::Net::HTTPResponse response; From cbf5f9a97e266d23df10ca6de914f784981ade73 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 9 Dec 2025 16:02:53 +0100 Subject: [PATCH 13/49] Allow custom CA certificate path for Azure Blob Storage connections This change enables Azure Blob Storage to use custom CA certificates for HTTPS connections, which is useful for testing environments or private deployments where certificates are not signed by public CAs. The implementation adds a configuration option that can be specified in Azure Blob Storage disk settings. When provided, this path is used to configure Curl's CAInfo option (Azure SDK uses Curl underneath) for all HTTPS connections to that Azure endpoint. Key changes: - Added field to RequestSettings struct - Updated getRequestSettings() to read ca_path from config - Modified getClientOptions() to set curl_options.CAInfo when curl_ca_path is provided - Endpoint-specific settings automatically supported through existing getRequestSettings() call Implementation details: - Uses Curl's built-in CAInfo option via Azure SDK's CurlTransportOptions - Simpler than S3 implementation as no HTTP layer modifications needed - Azure SDK handles HTTP/HTTPS layer, so we only configure Curl options This commit is based on the original patch: 0049-Custom_Azure_certificate_authority.patch Co-authored-by: Kevin Michel --- .../AzureBlobStorage/AzureBlobStorageCommon.cpp | 7 +++++++ .../AzureBlobStorage/AzureBlobStorageCommon.h | 1 + 2 files changed, 8 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 8d3b3696be12..42f6cb51fa56 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -390,6 +390,8 @@ BlobClientOptions getClientOptions( Azure::Core::Http::CurlTransportOptions curl_options; curl_options.NoSignal = true; curl_options.IPResolve = request_settings.curl_ip_resolve; + if (request_settings.curl_ca_path.has_value()) + curl_options.CAInfo = request_settings.curl_ca_path.value(); client_options.Transport.Transport = std::make_shared(curl_options); } @@ -584,6 +586,11 @@ std::unique_ptr getRequestSettings(const Poco::Util::AbstractCo else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value for option 'curl_ip_resolve': {}. Expected one of 'ipv4' or 'ipv6'", value); } + + if (config.has(config_prefix + ".ca_path")) + { + settings->curl_ca_path = config.getString(config_prefix + ".ca_path"); + } #endif return settings; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index e0bd961b2f6f..c17e462ccfb8 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -60,6 +60,7 @@ struct RequestSettings #if USE_AZURE_BLOB_STORAGE using CurlOptions = Azure::Core::Http::CurlTransportOptions; CurlOptions::CurlOptIPResolve curl_ip_resolve = CurlOptions::CURL_IPRESOLVE_WHATEVER; + std::optional curl_ca_path; #endif }; From 220d4ddfce71a466341735878b7b67c6d762d0af Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Wed, 10 Dec 2025 14:33:11 +0100 Subject: [PATCH 14/49] Fix default profile escape vulnerability This change prevents users from bypassing default profile restrictions by creating new users/roles that use less restrictive profiles (like admin profile). The security issue: Normally, ClickHouse prevents switching to settings profiles that are less restrictive than your current profile by verifying the new profiles against the constraints of the current profile. However, if a user has permission to create new users, they can create a user that directly uses the admin profile, then connect directly with this user. This way they can skip the verification that happens while switching profiles and escape the constraints of the default profile. The solution: Enforce that all created profiles must inherit from the default profile (or one of its descendants), unless the current user has the `allow_non_default_profile` setting enabled. This works because the admin profile is pre-created using users.xml and not via SQL, so it can already exist and be configured with `allow_non_default_profile` set to true, while the pre-created default profile has `allow_non_default_profile` set to false. Key changes: - Added `allow_non_default_profile` setting to control profile creation - Added methods to AccessControl and SettingsProfilesCache to check profile inheritance hierarchy - Added circular dependency detection when creating/altering profiles - Auto-inject default profile as parent if no parent is specified and user doesn't have `allow_non_default_profile` permission - Added validation in SettingsConstraints to ensure parent profiles inherit from default Co-authored-by: Kevin Michel --- src/Access/AccessControl.cpp | 17 +++++ src/Access/AccessControl.h | 4 ++ src/Access/SettingsConstraints.cpp | 13 ++++ src/Access/SettingsProfilesCache.cpp | 52 ++++++++++++++ src/Access/SettingsProfilesCache.h | 5 ++ src/Core/Settings.cpp | 3 + .../InterpreterCreateSettingsProfileQuery.cpp | 72 ++++++++++++++++++- 7 files changed, 164 insertions(+), 2 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 70818a25af16..e17d9b2d9320 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -666,6 +666,23 @@ void AccessControl::setDefaultProfileName(const String & default_profile_name) settings_profiles_cache->setDefaultProfileName(default_profile_name); } +std::optional AccessControl::getDefaultProfileId() const +{ + return settings_profiles_cache->getDefaultProfileId(); +} + + +bool AccessControl::isDefaultProfileOrDescendant(const UUID & profile_id) const +{ + return settings_profiles_cache->isDefaultProfileOrDescendant(profile_id); +} + + +bool AccessControl::isExpectedProfileOrDescendant(const UUID & profile_id, const UUID & expected_id) const +{ + return settings_profiles_cache->isExpectedProfileOrDescendant(profile_id, expected_id); +} + void AccessControl::setCustomSettingsPrefixes(const Strings & prefixes) { diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index c1e32fc7c467..7395680288c3 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -139,6 +139,10 @@ class AccessControl : public MultipleAccessStorage /// The default profile's settings are always applied before any other profile's. void setDefaultProfileName(const String & default_profile_name); + std::optional getDefaultProfileId() const; + bool isDefaultProfileOrDescendant(const UUID & profile_id) const; + bool isExpectedProfileOrDescendant(const UUID & profile_id, const UUID & expected_id) const; + /// Sets prefixes which should be used for custom settings. /// This function also enables custom prefixes to be used. void setCustomSettingsPrefixes(const Strings & prefixes); diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index d7e0114315e2..39b809a4a9c3 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -18,6 +18,7 @@ namespace Setting { extern const SettingsBool allow_ddl; extern const SettingsUInt64 readonly; + extern const SettingsBool allow_non_default_profile; } namespace ErrorCodes @@ -161,6 +162,18 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting if (SettingsProfileElements::isAllowBackupSetting(element.setting_name)) continue; + if (!current_settings[Setting::allow_non_default_profile]) + { + if (element.parent_profile.has_value()) + { + const auto & profile_id = element.parent_profile.value(); + if (!access_control->isDefaultProfileOrDescendant(profile_id)) + { + throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "All profiles must be the default profile or inherit from it."); + } + } + } + if (element.value) { SettingChange value(element.setting_name, *element.value); diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp index 4fb75e75e5a2..3fe455206cf4 100644 --- a/src/Access/SettingsProfilesCache.cpp +++ b/src/Access/SettingsProfilesCache.cpp @@ -99,6 +99,58 @@ void SettingsProfilesCache::setDefaultProfileName(const String & default_profile default_profile_id = it->second; } +std::optional SettingsProfilesCache::getDefaultProfileId() +{ + std::lock_guard lock{mutex}; + ensureAllProfilesRead(); + return default_profile_id; +} + + +bool SettingsProfilesCache::isDefaultProfileOrDescendant(const UUID & profile_id) +{ + std::lock_guard lock{mutex}; + ensureAllProfilesRead(); + if (default_profile_id.has_value()) + { + return isExpectedProfileOrDescendantLocked(profile_id, default_profile_id.value()); + } + return false; +} + + +bool SettingsProfilesCache::isExpectedProfileOrDescendant(const UUID & profile_id, const UUID & expected_id) +{ + std::lock_guard lock{mutex}; + ensureAllProfilesRead(); + return isExpectedProfileOrDescendantLocked(profile_id, expected_id); +} + + +bool SettingsProfilesCache::isExpectedProfileOrDescendantLocked(const UUID & profile_id, const UUID & expected_id) const +{ + /// `mutex` is already locked. + if (profile_id == expected_id) + { + return true; + } + const auto & profile = access_control.tryRead(profile_id); + if (profile) + { + for (const auto & element : profile->elements) + { + if (element.parent_profile.has_value()) + { + if (isExpectedProfileOrDescendantLocked(element.parent_profile.value(), expected_id)) + { + return true; + } + } + } + } + return false; +} + void SettingsProfilesCache::mergeSettingsAndConstraints() { diff --git a/src/Access/SettingsProfilesCache.h b/src/Access/SettingsProfilesCache.h index afc3c3e13a5c..6d844d543e26 100644 --- a/src/Access/SettingsProfilesCache.h +++ b/src/Access/SettingsProfilesCache.h @@ -22,6 +22,9 @@ class SettingsProfilesCache ~SettingsProfilesCache(); void setDefaultProfileName(const String & default_profile_name); + std::optional getDefaultProfileId(); + bool isDefaultProfileOrDescendant(const UUID & profile_id); + bool isExpectedProfileOrDescendant(const UUID & profile_id, const UUID & expected_id); std::shared_ptr getEnabledSettings( const UUID & user_id, @@ -38,6 +41,8 @@ class SettingsProfilesCache void mergeSettingsAndConstraints(); void mergeSettingsAndConstraintsFor(EnabledSettings & enabled) const; + bool isExpectedProfileOrDescendantLocked(const UUID & profile_id, const UUID & expected_id) const; + void substituteProfiles(SettingsProfileElements & elements, std::vector & profiles, std::vector & substituted_profiles, diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 5698a752f795..45f907c63c8e 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -2929,6 +2929,9 @@ Maximum size of query syntax tree in number of nodes after expansion of aliases \ DECLARE(UInt64, readonly, 0, R"( 0 - no read-only restrictions. 1 - only read requests, as well as changing explicitly allowed settings. 2 - only read requests, as well as changing settings, except for the 'readonly' setting. +)", 0) \ + DECLARE(Bool, allow_non_default_profile, true, R"( +When enabled, a user, role or settings profile can have a profile that is not the default profile or one of its descendants. )", 0) \ \ DECLARE(UInt64, max_rows_in_set, 0, R"( diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 5287edb1ee99..6f0ad686d5d5 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -1,9 +1,11 @@ +#include #include #include #include #include #include +#include #include #include #include @@ -14,9 +16,15 @@ namespace DB { +namespace Setting +{ + extern const SettingsBool allow_non_default_profile; +} + namespace ErrorCodes { extern const int ACCESS_ENTITY_ALREADY_EXISTS; + extern const int SETTING_CONSTRAINT_VIOLATION; } namespace @@ -61,14 +69,74 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() else getContext()->checkAccess(AccessType::CREATE_SETTINGS_PROFILE); + std::vector name_ids; + if (query.alter) + { + if (query.if_exists) + { + name_ids = access_control.find(query.names); + } + else + { + name_ids = access_control.getIDs(query.names); + } + } + + bool has_parent_profile = false; std::optional settings_from_query; if (query.alter_settings) settings_from_query = AlterSettingsProfileElements{*query.alter_settings, access_control}; else if (query.settings) settings_from_query = AlterSettingsProfileElements{SettingsProfileElements(*query.settings, access_control)}; - if (settings_from_query && !query.attach) - getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::PROFILE); + if (settings_from_query) + + { + for (const auto & element : settings_from_query->add_settings) + { + if (element.parent_profile.has_value()) + { + has_parent_profile = true; + for (const auto & name_id: name_ids) + { + if (access_control.isExpectedProfileOrDescendant(element.parent_profile.value(), name_id)) + { + throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Inherited profiles may not have circular dependencies"); + } + } + } + } + // This checks that each defined parent is either default or inheriting from it + if (!query.attach) + getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::PROFILE); + } + + // If the query is a create (= not an alter), then it must either be the default + // or have a parent profile (which has been checked to be to inherit from default). + // If the query is an alter and it has settings change, same thing applies. + // A query can be an alter without settings change if it's a noop or just renaming the profile itself. + // Instead of rejecting a query without a parent, we auto-inject the default profile as a parent. + if (!query.alter || settings_from_query) + { + if (!has_parent_profile && !getContext()->getSettingsRef()[Setting::allow_non_default_profile]) + { + bool has_default_profile = false; + const auto & default_profile_id = access_control.getDefaultProfileId(); + if (!default_profile_id.has_value()) + throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Cannot alter or create profiles if default profile is not configured."); + for (const auto & name_id : name_ids) + if (name_id == default_profile_id.value()) + has_default_profile = true; + if (has_default_profile and query.names.size() > 1) + throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Cannot alter the default and other profiles simultaneously."); + SettingsProfileElement default_parent; + default_parent.parent_profile = default_profile_id; + default_parent.setting_name = "profile"; + if (!settings_from_query.has_value()) + settings_from_query = AlterSettingsProfileElements(); + settings_from_query->add_settings.insert(settings_from_query->add_settings.begin(), default_parent); + } + } if (!query.cluster.empty()) { From 1874cd8f9444eb084c68a05d816f850e8f26e240 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Fri, 12 Dec 2025 10:43:06 +0100 Subject: [PATCH 15/49] Allow delegating S3 signature to a separate process This commit adds support for delegating AWS S3 signature generation to an external HTTP service. This enables use cases where signature generation needs to be controlled by a proxy or external service for security, compliance, or monitoring purposes. The process is accessed over HTTP and the URL to access it can be configured with the signature_delegation_url parameter in the S3 disk configuration. ClickHouse will make a POST request with a JSON body: { "canonicalRequest": "PUT\n..." } And expects a JSON response with status 200 and this structure: { "signature": "01234567890abcdef..." } The canonical request matches the format defined by AWS for signatures and contains all the required information (path, host, operation...) to let the proxy decide if the request is allowed and can be signed. Changes: - Added AWSAuthV4DelegatedSigner class that delegates signature generation to an external HTTP service - Updated S3Client to use AWSAuthSignerProvider-based constructor (compatible with newer AWS SDK) - Added signature_delegation_url configuration parameter to S3 disk settings - Updated AWS SDK submodule to aiven/clickhouse-v25.8.12.129 branch which includes the necessary SDK changes for delegated signatures - Updated all S3 client creation sites to pass the new parameter This commit was adapted from the original patch to work with the newer AWS SDK version (1.7.321) used in ClickHouse v25.8.12.129, which requires using AWSAuthSignerProvider instead of the older constructor signature. Co-authored-by: Kevin Michel --- .gitmodules | 3 +- contrib/aws | 2 +- src/Backups/BackupIO_S3.cpp | 3 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 3 +- src/Databases/DataLake/GlueCatalog.cpp | 4 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 4 +- src/IO/S3/AWSAuthV4DelegatedSigner.cpp | 107 +++++++++++++++++++ src/IO/S3/AWSAuthV4DelegatedSigner.h | 37 +++++++ src/IO/S3/Client.cpp | 82 +++++++++++--- src/IO/S3/Client.h | 12 +-- src/IO/S3/Credentials.cpp | 11 +- src/IO/S3/PocoHTTPClient.cpp | 2 + src/IO/S3/PocoHTTPClient.h | 5 +- src/IO/S3/tests/gtest_aws_s3_client.cpp | 6 +- src/IO/S3AuthSettings.cpp | 1 + src/IO/tests/gtest_readbuffer_s3.cpp | 7 +- src/IO/tests/gtest_writebuffer_s3.cpp | 4 +- 17 files changed, 257 insertions(+), 36 deletions(-) create mode 100644 src/IO/S3/AWSAuthV4DelegatedSigner.cpp create mode 100644 src/IO/S3/AWSAuthV4DelegatedSigner.h diff --git a/.gitmodules b/.gitmodules index 1c7304defc3c..a75556abd105 100644 --- a/.gitmodules +++ b/.gitmodules @@ -87,7 +87,8 @@ url = https://github.com/ClickHouse/grpc [submodule "contrib/aws"] path = contrib/aws - url = https://github.com/ClickHouse/aws-sdk-cpp + url = https://github.com/aiven/aws-sdk-cpp + branch = aiven/clickhouse-v25.8.12.129 [submodule "contrib/aws-c-event-stream"] path = contrib/aws-c-event-stream url = https://github.com/awslabs/aws-c-event-stream diff --git a/contrib/aws b/contrib/aws index a86b913abc27..39c331979c58 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit a86b913abc2795ee23941b24dd51e862214ec6b0 +Subproject commit 39c331979c58146578dfedae56449cebf177125a diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 6d88f4294352..f12ba71bcd6a 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -151,7 +151,8 @@ class S3BackupClientCreator /* opt_disk_name = */ {}, request_settings.get_request_throttler, request_settings.put_request_throttler, - s3_uri.uri.getScheme()); + s3_uri.uri.getScheme(), + /* signature_delegation_url = */ ""); client_configuration.endpointOverride = s3_uri.endpoint; client_configuration.maxConnections = static_cast(global_settings[Setting::s3_max_connections]); diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 74a0c1289ebd..6811e97c2552 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -127,7 +127,8 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}, - new_uri.uri.getScheme()); + new_uri.uri.getScheme(), + /* signature_delegation_url = */ ""); client_configuration.endpointOverride = new_uri.endpoint; diff --git a/src/Databases/DataLake/GlueCatalog.cpp b/src/Databases/DataLake/GlueCatalog.cpp index 6cb14ae965e7..110320690f28 100644 --- a/src/Databases/DataLake/GlueCatalog.cpp +++ b/src/Databases/DataLake/GlueCatalog.cpp @@ -125,7 +125,9 @@ GlueCatalog::GlueCatalog( /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, /* get_request_throttler = */ nullptr, - /* put_request_throttler = */ nullptr); + /* put_request_throttler = */ nullptr, + /* protocol = */ "https", + /* signature_delegation_url = */ ""); Aws::Glue::GlueClientConfiguration client_configuration; client_configuration.maxConnections = static_cast(global_settings[DB::Setting::s3_max_connections]); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index c486c8d3b5b2..29e786f13351 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -60,6 +60,7 @@ namespace S3AuthSetting extern const S3AuthSettingsString service_account; extern const S3AuthSettingsString metadata_service; extern const S3AuthSettingsString request_token_path; + extern const S3AuthSettingsString signature_delegation_url; } namespace ErrorCodes @@ -127,7 +128,8 @@ getClient(const S3::URI & url, const S3Settings & settings, ContextPtr context, opt_disk_name, request_settings.get_request_throttler, request_settings.put_request_throttler, - url.uri.getScheme()); + url.uri.getScheme(), + auth_settings[S3AuthSetting::signature_delegation_url]); client_configuration.connectTimeoutMs = auth_settings[S3AuthSetting::connect_timeout_ms]; client_configuration.requestTimeoutMs = auth_settings[S3AuthSetting::request_timeout_ms]; diff --git a/src/IO/S3/AWSAuthV4DelegatedSigner.cpp b/src/IO/S3/AWSAuthV4DelegatedSigner.cpp new file mode 100644 index 000000000000..595aed4295d2 --- /dev/null +++ b/src/IO/S3/AWSAuthV4DelegatedSigner.cpp @@ -0,0 +1,107 @@ +#include "AWSAuthV4DelegatedSigner.h" + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::S3 +{ + +AWSAuthV4DelegatedSigner::AWSAuthV4DelegatedSigner( + const Aws::String & signatureDelegationUrl, + const std::shared_ptr & credentialsProvider, + const char * serviceName, + const Aws::String & region, + PayloadSigningPolicy signingPolicy, + bool urlEscapePath, + Aws::Auth::AWSSigningAlgorithm signingAlgorithm) + : Aws::Client::AWSAuthV4Signer(credentialsProvider, serviceName, region, signingPolicy, urlEscapePath, signingAlgorithm) + , signature_delegation_url(signatureDelegationUrl) + , logger(&Poco::Logger::get("AWSAuthV4DelegatedSigner")) +{ +} + +Aws::String AWSAuthV4DelegatedSigner::GenerateSignature( + const Aws::String & canonicalRequestString, + const Aws::String & date, + const Aws::String & simpleDate, + const Aws::String & signingRegion, + const Aws::String & signingServiceName, + const Aws::Auth::AWSCredentials & credentials) const +{ + if (signature_delegation_url.empty()) + { + return Aws::Client::AWSAuthV4Signer::GenerateSignature( + canonicalRequestString, date, simpleDate, signingRegion, signingServiceName, credentials + ); + } + Poco::URI url(signature_delegation_url); + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request.setHost(url.getHost()); + + Poco::JSON::Object request_json; + request_json.set("canonicalRequest", canonicalRequestString); + std::ostringstream request_string_stream; + request_string_stream.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(request_json, request_string_stream); + request.setContentLength(request_string_stream.tellp()); + request.setContentType("application/json"); + + auto timeouts = ConnectionTimeouts() + .withConnectionTimeout(30) + .withSendTimeout(30) + .withReceiveTimeout(30) + // Since the proxy is on the same host, we don't need to keep the + // connection alive. The OS proxy has a low keep-alive timeout, if the + // server closes the connection early we can get NoMessageException + // errors. + .withHTTPKeepAliveTimeout(0); + + std::string response_body; + auto session = makeHTTPSession(HTTPConnectionGroupType::DISK, url, timeouts); + try + { + auto & request_body_stream = session->sendRequest(request); + request_body_stream << request_string_stream.str(); + Poco::Net::HTTPResponse response; + auto & response_body_stream = session->receiveResponse(response); + assertResponseIsOk(request.getURI(), response, response_body_stream, /* allow_redirects= */ false); + Poco::StreamCopier::copyToString(response_body_stream, response_body); + } + catch (const Poco::Exception &) + { + auto message = getCurrentExceptionMessage(true); + LOG_ERROR(logger, "Request for signature delegation failed. Error: {}", message); + return ""; + } + + try + { + Poco::JSON::Parser parser; + auto response_json = parser.parse(response_body).extract(); + return response_json->getValue("signature"); + } + catch (const Poco::Exception &) + { + auto message = getCurrentExceptionMessage(true); + LOG_ERROR(logger, "Parsing signature delegation response failed. Error {}", message); + return ""; + } +} + +} + +#endif + diff --git a/src/IO/S3/AWSAuthV4DelegatedSigner.h b/src/IO/S3/AWSAuthV4DelegatedSigner.h new file mode 100644 index 000000000000..7adce70bab12 --- /dev/null +++ b/src/IO/S3/AWSAuthV4DelegatedSigner.h @@ -0,0 +1,37 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 +#include +#include +#include + +namespace DB::S3 +{ +class AWSAuthV4DelegatedSigner : public Aws::Client::AWSAuthV4Signer +{ +public: + AWSAuthV4DelegatedSigner( + const Aws::String & signatureDelegationUrl, + const std::shared_ptr & credentialsProvider, + const char * serviceName, + const Aws::String & region, + PayloadSigningPolicy signingPolicy = PayloadSigningPolicy::RequestDependent, + bool urlEscapePath = true, + Aws::Auth::AWSSigningAlgorithm signingAlgorithm = Aws::Auth::AWSSigningAlgorithm::SIGV4); + + Aws::String GenerateSignature( + const Aws::String & canonicalRequestString, + const Aws::String & date, + const Aws::String & simpleDate, + const Aws::String & signingRegion, + const Aws::String & signingServiceName, + const Aws::Auth::AWSCredentials& credentials) const override; +private: + Aws::String signature_delegation_url; + Poco::Logger * logger; +}; +} + +#endif diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 31eb0789babc..854888ebe652 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -26,7 +26,9 @@ #include #include #include +#include #include +#include #include #include @@ -216,12 +218,12 @@ std::unique_ptr Client::create( ServerSideEncryptionKMSConfig sse_kms_config_, const std::shared_ptr & credentials_provider, const PocoHTTPClientConfiguration & client_configuration, - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - const ClientSettings & client_settings) + const ClientSettings & client_settings, + const String & signature_delegation_url) { verifyClientConfiguration(client_configuration); return std::unique_ptr( - new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, client_settings)); + new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, client_settings, signature_delegation_url)); } std::unique_ptr Client::clone() const @@ -250,17 +252,66 @@ ProviderType deduceProviderType(const std::string & url) } +namespace +{ + +std::shared_ptr createSignerProvider( + const std::shared_ptr & credentials_provider, + const PocoHTTPClientConfiguration & client_configuration, + const ClientSettings & client_settings, + const String & signature_delegation_url) +{ + // Determine region for signer + Aws::String region = client_configuration.region; + if (region.empty()) + region = Aws::Region::US_EAST_1; + + // Determine payload signing policy + Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy signing_policy = + client_settings.is_s3express_bucket + ? Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::RequestDependent + : Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never; + + // If signature delegation is enabled, create delegated signer + if (!signature_delegation_url.empty()) + { + auto delegated_signer = std::make_shared( + signature_delegation_url, + credentials_provider, + "s3", + region, + signing_policy, + true, // urlEscapePath + Aws::Auth::AWSSigningAlgorithm::SIGV4); + + return std::make_shared(delegated_signer); + } + else + { + // Create default signer provider with regular signer + return std::make_shared( + credentials_provider, + "s3", + region, + signing_policy); + } +} + +} + Client::Client( size_t max_redirects_, ServerSideEncryptionKMSConfig sse_kms_config_, const std::shared_ptr & credentials_provider_, const PocoHTTPClientConfiguration & client_configuration_, - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads_, - const ClientSettings & client_settings_) - : Aws::S3::S3Client(credentials_provider_, client_configuration_, sign_payloads_, client_settings_.use_virtual_addressing) + const ClientSettings & client_settings_, + const String & signature_delegation_url_) + : Aws::S3::S3Client( + createSignerProvider(credentials_provider_, client_configuration_, client_settings_, signature_delegation_url_), + Aws::MakeShared(Aws::S3::S3Client::ALLOCATION_TAG), + client_configuration_) , credentials_provider(credentials_provider_) , client_configuration(client_configuration_) - , sign_payloads(sign_payloads_) , client_settings(client_settings_) , max_redirects(max_redirects_) , sse_kms_config(std::move(sse_kms_config_)) @@ -304,12 +355,14 @@ Client::Client( Client::Client( const Client & other, const PocoHTTPClientConfiguration & client_configuration_) - : Aws::S3::S3Client(other.credentials_provider, client_configuration_, other.sign_payloads, - other.client_settings.use_virtual_addressing) + : Aws::S3::S3Client( + createSignerProvider(other.credentials_provider, client_configuration_, other.client_settings, + client_configuration_.signature_delegation_url), + Aws::MakeShared(Aws::S3::S3Client::ALLOCATION_TAG), + client_configuration_) , initial_endpoint(other.initial_endpoint) , credentials_provider(other.credentials_provider) , client_configuration(client_configuration_) - , sign_payloads(other.sign_payloads) , client_settings(other.client_settings) , explicit_region(other.explicit_region) , detect_region(other.detect_region) @@ -1206,9 +1259,8 @@ std::unique_ptr ClientFactory::create( // NOLINT std::move(sse_kms_config), credentials_provider, client_configuration, // Client configuration. - client_settings.is_s3express_bucket ? Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::RequestDependent - : Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - client_settings); + client_settings, + client_configuration.signature_delegation_url); } PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT @@ -1224,7 +1276,8 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT std::optional opt_disk_name, const ThrottlerPtr & get_request_throttler, const ThrottlerPtr & put_request_throttler, - const String & protocol) + const String & protocol, + const String & signature_delegation_url) { auto context = Context::getGlobalContextInstance(); chassert(context); @@ -1248,6 +1301,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT context->getGlobalContext()->getSettingsRef()[Setting::s3_use_adaptive_timeouts], get_request_throttler, put_request_throttler, + signature_delegation_url, error_report); config.scheme = Aws::Http::SchemeMapper::FromString(protocol.c_str()); diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 6d1b738840bf..8b414f50459f 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -127,8 +127,8 @@ class Client : private Aws::S3::S3Client ServerSideEncryptionKMSConfig sse_kms_config_, const std::shared_ptr & credentials_provider, const PocoHTTPClientConfiguration & client_configuration, - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - const ClientSettings & client_settings); + const ClientSettings & client_settings, + const String & signature_delegation_url = ""); std::unique_ptr clone() const; @@ -250,8 +250,8 @@ class Client : private Aws::S3::S3Client ServerSideEncryptionKMSConfig sse_kms_config_, const std::shared_ptr & credentials_provider_, const PocoHTTPClientConfiguration & client_configuration, - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - const ClientSettings & client_settings_); + const ClientSettings & client_settings_, + const String & signature_delegation_url = ""); private: Client( @@ -307,7 +307,6 @@ class Client : private Aws::S3::S3Client String initial_endpoint; std::shared_ptr credentials_provider; PocoHTTPClientConfiguration client_configuration; - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads; ClientSettings client_settings; std::string explicit_region; @@ -363,7 +362,8 @@ class ClientFactory std::optional opt_disk_name, const ThrottlerPtr & get_request_throttler, const ThrottlerPtr & put_request_throttler, - const String & protocol = "https"); + const String & protocol = "https", + const String & signature_delegation_url = ""); private: ClientFactory(); diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 2986435d5451..8920f5d63bf3 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -745,7 +745,9 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.for_disk_s3, configuration.opt_disk_name, configuration.get_request_throttler, - configuration.put_request_throttler); + configuration.put_request_throttler, + /* protocol = */ "https", + /* signature_delegation_url = */ ""); AddProvider(std::make_shared(aws_client_configuration, credentials_configuration.expiration_window_seconds, credentials_configuration.kms_role_arn)); } @@ -764,7 +766,9 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.for_disk_s3, configuration.opt_disk_name, configuration.get_request_throttler, - configuration.put_request_throttler); + configuration.put_request_throttler, + /* protocol = */ "https", + /* signature_delegation_url = */ ""); AddProvider(std::make_shared( std::move(aws_client_configuration), credentials_configuration.expiration_window_seconds)); } @@ -819,7 +823,8 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.opt_disk_name, configuration.get_request_throttler, configuration.put_request_throttler, - Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP)); + Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP), + /* signature_delegation_url = */ ""); /// See MakeDefaultHTTPResourceClientConfiguration(). /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 0a845eea7e4f..f6d4b3b6e99c 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -110,6 +110,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, + const String & signature_delegation_url_, std::function error_report_) : per_request_configuration(per_request_configuration_) , force_region(force_region_) @@ -124,6 +125,7 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( , opt_disk_name(opt_disk_name_) , get_request_throttler(get_request_throttler_) , put_request_throttler(put_request_throttler_) + , signature_delegation_url(signature_delegation_url_) , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 6074f2125a62..e74430654a9a 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -20,6 +20,7 @@ #include #include #include +#include #include @@ -42,7 +43,7 @@ class ClientFactory; class PocoHTTPClient; -struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration +struct PocoHTTPClientConfiguration : public Aws::S3::S3ClientConfiguration { struct RetryStrategy { @@ -70,6 +71,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration String service_account; String metadata_service; String request_token_path; + String signature_delegation_url; /// See PoolBase::BehaviourOnLimit bool s3_use_adaptive_timeouts = true; @@ -100,6 +102,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration bool s3_use_adaptive_timeouts_, const ThrottlerPtr & get_request_throttler_, const ThrottlerPtr & put_request_throttler_, + const String & signature_delegation_url_, std::function error_report_); /// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization. diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 1c7b2c17b834..d66cbdb6fb70 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -145,7 +145,8 @@ void testServerSideEncryption( /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}, - uri.uri.getScheme()); + uri.uri.getScheme(), + /* signature_delegation_url = */ ""); client_configuration.endpointOverride = uri.endpoint; @@ -422,7 +423,8 @@ TEST(IOTestAwsS3Client, AssumeRole) /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}, - "http"); + "http", + /* signature_delegation_url = */ ""); client_configuration.endpointOverride = uri.endpoint; client_configuration.retryStrategy = std::make_shared(); diff --git a/src/IO/S3AuthSettings.cpp b/src/IO/S3AuthSettings.cpp index 708909712364..32382feba5fc 100644 --- a/src/IO/S3AuthSettings.cpp +++ b/src/IO/S3AuthSettings.cpp @@ -45,6 +45,7 @@ namespace ErrorCodes DECLARE(String, service_account, "", "", 0) \ DECLARE(String, metadata_service, "", "", 0) \ DECLARE(String, request_token_path, "", "", 0) \ + DECLARE(String, signature_delegation_url, "", "", 0) #define CLIENT_SETTINGS_LIST(M, ALIAS) \ CLIENT_SETTINGS(M, ALIAS) \ diff --git a/src/IO/tests/gtest_readbuffer_s3.cpp b/src/IO/tests/gtest_readbuffer_s3.cpp index 616333cf8309..13a6bde4b107 100644 --- a/src/IO/tests/gtest_readbuffer_s3.cpp +++ b/src/IO/tests/gtest_readbuffer_s3.cpp @@ -65,9 +65,10 @@ struct ClientFake : DB::S3::Client {}, {}, {}, - "http"), - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - DB::S3::ClientSettings()) + "http", + /* signature_delegation_url = */ ""), + DB::S3::ClientSettings(), + /* signature_delegation_url = */ "") { } diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 74702949750b..8d8a7997e39a 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -252,7 +252,9 @@ struct Client : DB::S3::Client /* for_disk_s3 = */ false, /* opt_disk_name = */ {}, /* get_request_throttler = */ {}, - /* put_request_throttler = */ {}); + /* put_request_throttler = */ {}, + /* protocol = */ "https", + /* signature_delegation_url = */ ""); } void setInjectionModel(std::shared_ptr injections_) From cd35e550bdf70cf431d3b51220efc3cd54eee09d Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sat, 13 Dec 2025 16:44:11 +0100 Subject: [PATCH 16/49] Allow delegating Azure signature to a separate process The process is accessed over HTTP and the URL to access it can be configured with the signature_delegation_url parameter in the Azure disk configuration. ClickHouse will make a POST request with a json body: { "stringToSign": "..." } And expects a JSON response with status 200 and this structure: { "signature": "01234567890abcdefg..." } The string to sign matches the format defined by Azure for signatures. The canonical request also contains all the required information (path, operation...) to let the proxy decide if the request is allowed and can be signed. It's also enough information to know the cost of the request. This commit was applied from the patch file 0067-Delegated_signature_azure.patch and adapted to work with ClickHouse v25.8.12.129 codebase structure. Changes include: - Updated Azure SDK submodule to aiven/azure-sdk-for-cpp fork with branch aiven/clickhouse-v25.8.12.129 containing the SDK modifications (virtual GetSignature method in SharedKeyPolicy) - Added AzureDelegatedKeyPolicy class that extends SharedKeyPolicy to delegate signature generation via HTTP POST requests - Added account_name and signature_delegation_url fields to RequestSettings for configuration - Added delegated_signature flag to ConnectionParams to control client creation behavior - Updated all ConnectionParams creation sites to include the delegated_signature flag - Modified getClientOptions to inject AzureDelegatedKeyPolicy into PerRetryPolicies when signature delegation is enabled Co-authored-by: Kevin Michel --- .gitmodules | 3 +- contrib/azure | 2 +- .../registerBackupEngineAzureBlobStorage.cpp | 3 + .../AzureBlobStorageCommon.cpp | 23 ++++++ .../AzureBlobStorage/AzureBlobStorageCommon.h | 5 ++ .../AzureDelegatedKeyPolicy.cpp | 73 +++++++++++++++++++ .../AzureDelegatedKeyPolicy.h | 38 ++++++++++ .../AzureBlobStorage/AzureObjectStorage.cpp | 1 + .../AzureObjectStorageConnectionInfo.cpp | 1 + .../ObjectStorages/ObjectStorageFactory.cpp | 1 + 10 files changed, 148 insertions(+), 2 deletions(-) create mode 100644 src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.cpp create mode 100644 src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.h diff --git a/.gitmodules b/.gitmodules index a75556abd105..bc363d734f90 100644 --- a/.gitmodules +++ b/.gitmodules @@ -215,7 +215,8 @@ url = https://github.com/ClickHouse/hive-metastore [submodule "contrib/azure"] path = contrib/azure - url = https://github.com/ClickHouse/azure-sdk-for-cpp + url = https://github.com/aiven/azure-sdk-for-cpp + branch = aiven/clickhouse-v25.8.12.129 [submodule "contrib/minizip-ng"] path = contrib/minizip-ng url = https://github.com/zlib-ng/minizip-ng diff --git a/contrib/azure b/contrib/azure index 0f7a2013f7d7..3278daf75acf 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 0f7a2013f7d79058047fc4bd35e94d20578c0d2b +Subproject commit 3278daf75acfb195b7746ef50c82c0a35abb5e02 diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index b7e880c90160..e2ed50bed248 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -70,6 +70,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), .client_options = AzureBlobStorage::getClientOptions(params.context, params.context->getSettingsRef(), *request_settings, /*for_disk=*/ true), + .delegated_signature = AzureBlobStorage::isDelegatedSignature(*request_settings), }; if (args.size() > 1) @@ -89,6 +90,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) AzureBlobStorage::processURL(connection_url, container_name, connection_params.endpoint, connection_params.auth_method); connection_params.client_options = AzureBlobStorage::getClientOptions(params.context, params.context->getSettingsRef(), *request_settings, /*for_disk=*/ true); + connection_params.delegated_signature = AzureBlobStorage::isDelegatedSignature(*request_settings); } else if (args.size() == 5) { @@ -101,6 +103,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) connection_params.auth_method = std::make_shared(account_name, account_key); connection_params.client_options = AzureBlobStorage::getClientOptions(params.context, params.context->getSettingsRef(), *request_settings, /*for_disk=*/ true); + connection_params.delegated_signature = AzureBlobStorage::isDelegatedSignature(*request_settings); } else { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 42f6cb51fa56..8a4877244dab 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -1,4 +1,5 @@ #include +#include #if USE_AZURE_BLOB_STORAGE @@ -208,6 +209,11 @@ std::unique_ptr ConnectionParams::createForContainer() const return std::make_unique(std::move(raw_client), endpoint.prefix); } + if (delegated_signature) { + RawContainerClient raw_client{endpoint.getContainerEndpoint(), client_options}; + return std::make_unique(std::move(raw_client), endpoint.prefix); + } + return std::visit([this](const T & auth) { if constexpr (std::is_same_v) @@ -326,6 +332,10 @@ AuthMethod getAuthMethod(const Poco::Util::AbstractConfiguration & config, const return getManagedIdentityCredential(); } +bool isDelegatedSignature(const RequestSettings &settings) { + return settings.account_name.has_value() && settings.signature_delegation_url.has_value(); +} + BlobClientOptions getClientOptions( const ContextPtr & context, const Settings & settings, @@ -340,6 +350,14 @@ BlobClientOptions getClientOptions( client_options.Retry = retry_options; client_options.ClickhouseOptions = Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=for_disk}; + if (request_settings.account_name.has_value() && request_settings.signature_delegation_url.has_value()) + { + auto storage_shared_key_credential + = std::make_shared(request_settings.account_name.value(), /* account_key= */ "ignored"); + client_options.PerRetryPolicies.emplace_back( + std::make_unique(storage_shared_key_credential, request_settings.signature_delegation_url.value())); + } + if (settings[Setting::azure_sdk_use_native_client]) { ThrottlerPtr get_request_throttler; @@ -591,6 +609,11 @@ std::unique_ptr getRequestSettings(const Poco::Util::AbstractCo { settings->curl_ca_path = config.getString(config_prefix + ".ca_path"); } + + if (config.has(config_prefix + ".account_name") && config.has(config_prefix + ".signature_delegation_url")) { + settings->account_name = config.getString(config_prefix + ".account_name"); + settings->signature_delegation_url = config.getString(config_prefix + ".signature_delegation_url"); + } #endif return settings; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index c17e462ccfb8..a75ac00c75f0 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -61,6 +61,9 @@ struct RequestSettings using CurlOptions = Azure::Core::Http::CurlTransportOptions; CurlOptions::CurlOptIPResolve curl_ip_resolve = CurlOptions::CURL_IPRESOLVE_WHATEVER; std::optional curl_ca_path; + + std::optional account_name; + std::optional signature_delegation_url; #endif }; @@ -142,6 +145,7 @@ struct ConnectionParams Endpoint endpoint; AuthMethod auth_method; BlobClientOptions client_options; + bool delegated_signature; String getContainer() const { return endpoint.container_name; } String getConnectionURL() const; @@ -161,6 +165,7 @@ BlobClientOptions getClientOptions( bool for_disk); AuthMethod getAuthMethod(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); +bool isDelegatedSignature(const RequestSettings & settings); #endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.cpp new file mode 100644 index 000000000000..5f6bb406e76f --- /dev/null +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.cpp @@ -0,0 +1,73 @@ +#include + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB { + +std::string AzureDelegatedKeyPolicy::GetSignature(const std::string& string_to_sign) const +{ + Poco::URI url(signature_delegation_url); + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request.setHost(url.getHost()); + + Poco::JSON::Object request_json; + request_json.set("stringToSign", string_to_sign); + std::ostringstream request_string_stream; + request_string_stream.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(request_json, request_string_stream); + request.setContentLength(request_string_stream.tellp()); + request.setContentType("application/json"); + + auto timeouts = ConnectionTimeouts() + .withConnectionTimeout(30) + .withSendTimeout(30) + .withReceiveTimeout(30) + // Since the proxy is on the same host, we don't need to keep the + // connection alive. The OS proxy has a low keep-alive timeout, if the + // server closes the connection early we can get NoMessageException + // errors. + .withHTTPKeepAliveTimeout(0); + + + std::string response_body; + auto session = makeHTTPSession(HTTPConnectionGroupType::DISK, url, timeouts); + try + { + auto & request_body_stream = session->sendRequest(request); + request_body_stream << request_string_stream.str(); + Poco::Net::HTTPResponse response; + auto & response_body_stream = session->receiveResponse(response); + assertResponseIsOk(request.getURI(), response, response_body_stream, /* allow_redirects= */ false); + Poco::StreamCopier::copyToString(response_body_stream, response_body); + } + catch (const Poco::Exception & e) + { + session->attachSessionData(e.message()); + throw Exception(Exception::CreateFromPocoTag{}, e); + } + + try + { + Poco::JSON::Parser parser; + auto response_json = parser.parse(response_body).extract(); + return response_json->getValue("signature"); + } + catch (const Poco::Exception & e) + { + throw Exception(Exception::CreateFromPocoTag{}, e); + } +} + +} + +#endif + diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.h new file mode 100644 index 000000000000..1558663d4601 --- /dev/null +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureDelegatedKeyPolicy.h @@ -0,0 +1,38 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include + +namespace DB +{ + +class AzureDelegatedKeyPolicy : public Azure::Storage::_internal::SharedKeyPolicy +{ +public: + explicit AzureDelegatedKeyPolicy(const std::shared_ptr &credential, const std::string & signature_delegation_url_) + : SharedKeyPolicy(credential) + , signature_delegation_url(signature_delegation_url_) + { + } + + ~AzureDelegatedKeyPolicy() override = default; + + std::unique_ptr Clone() const override + { + return std::make_unique(m_credential, signature_delegation_url); + } + +protected: + std::string GetSignature(const std::string& string_to_sign) const override; + + std::string signature_delegation_url; +}; + +} + +#endif + diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 83ca6950da69..a4d77aac1087 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -377,6 +377,7 @@ void AzureObjectStorage::applyNewSettings( .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), .client_options = AzureBlobStorage::getClientOptions(context, context->getSettingsRef(), *settings.get(), is_client_for_disk), + .delegated_signature = AzureBlobStorage::isDelegatedSignature(*settings.get()), }; auto new_client = AzureBlobStorage::getContainerClient(params, /*readonly=*/ true); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp index 4ba17b29d842..a496550657dc 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp @@ -118,6 +118,7 @@ std::unique_ptr AzureObjectStorageConnectionInfo::create std::shared_ptr AzureObjectStorageConnectionInfo::makeClient(const AzureClientInfo & info) { auto params = AzureBlobStorage::ConnectionParams{}; + params.delegated_signature = false; AzureBlobStorage::processURL(info.endpoint, info.container_name, params.endpoint, params.auth_method); auto global_context = Context::getGlobalContextInstance(); diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 67c09c0e0a59..6b1009c092d6 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -304,6 +304,7 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), .client_options = AzureBlobStorage::getClientOptions(context, context->getSettingsRef(), *azure_settings, /*for_disk=*/ true), + .delegated_signature = AzureBlobStorage::isDelegatedSignature(*azure_settings), }; return createObjectStorage( From 56ae31246fa5d3d3469e1c2a0de1cfbcff491926 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sat, 13 Dec 2025 16:50:09 +0100 Subject: [PATCH 17/49] Enforce SSL in the MySQL handler This commit enforces SSL/TLS encryption for all MySQL protocol connections to ClickHouse. Previously, clients could connect without SSL and communicate in plaintext. Now, any connection attempt without SSL is rejected with an error message. When a client attempts to connect without SSL: - The connection is immediately rejected - An error packet (MySQL error code 3159) is sent with a clear message - The connection is closed Error message: "SSL support for MySQL TCP protocol is required. If using the MySQL CLI client, please connect with --ssl-mode=REQUIRED." This is a security hardening change that ensures all MySQL protocol traffic is encrypted, protecting credentials and data in transit. This helps meet compliance requirements and prevents man-in-the-middle attacks. This commit was applied from the patch file 0071-MySQL_enforce_SSL.patch Co-authored-by: Joe Lynch --- src/Server/MySQLHandler.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 58e7acaa2dc3..f73610ee9361 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -403,15 +403,13 @@ void MySQLHandler::finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResp } else { - /// Reading rest of HandshakeResponse. + /// Send error packet and close connection. packet_size = PACKET_HEADER_SIZE + payload_size; - WriteBufferFromOwnString buf_for_handshake_response; - buf_for_handshake_response.write(buf.data(), pos); - copyData(*packet_endpoint->in, buf_for_handshake_response, packet_size - pos); - ReadBufferFromString payload(buf_for_handshake_response.str()); - payload.ignore(PACKET_HEADER_SIZE); - packet.readPayloadWithUnpacked(payload); + packet_endpoint->in->ignore(packet_size - pos); + static constexpr const char * error_msg = "SSL support for MySQL TCP protocol is required. If using the MySQL CLI client, please connect with --ssl-mode=REQUIRED."; packet_endpoint->sequence_id++; + packet_endpoint->sendPacket(ERRPacket(3159, "HY000", error_msg), true); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, error_msg); } } From 97c535232ec7b50ac66567b7f209dbd5914dd7b3 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sat, 13 Dec 2025 16:55:44 +0100 Subject: [PATCH 18/49] Enforce HTTPS for URL storage and HTTPDictionarySource This commit enforces HTTPS/TLS encryption for all HTTP/HTTPS connections in two ClickHouse features: 1. HTTP Dictionary Sources - dictionaries loaded from remote URLs 2. URL Storage - tables that read/write data from remote URLs Previously, both features allowed unencrypted HTTP connections, which exposed credentials, queries, and data to potential interception. Now, any attempt to use HTTP (non-HTTPS) URLs is rejected with a clear error message. Changes include: - Changed HTTPDictionarySource::Configuration::url from std::string to Poco::URI for better type safety and scheme validation - Added HTTPS validation in HTTPDictionarySource registration: throws UNSUPPORTED_METHOD error if scheme is not "https" - Added HTTPS validation in StorageURL constructor: throws BAD_ARGUMENTS error if scheme is not "https" - Simplified code by removing redundant Poco::URI object creations and using configuration.url directly throughout Security impact: - Prevents unencrypted transmission of dictionary data, table data, credentials, and query parameters - Protects against man-in-the-middle attacks - Helps meet compliance requirements (PCI-DSS, HIPAA, GDPR) - Fails fast with clear error messages if HTTP is used Breaking change: Existing configurations using http:// URLs will fail and must be updated to use https:// URLs. This commit was applied from the patch file 0091-HTTP-dictionary-source.patch Co-authored-by: Joe Lynch --- src/Dictionaries/HTTPDictionarySource.cpp | 27 ++++++++++------------- src/Dictionaries/HTTPDictionarySource.h | 2 +- src/Storages/StorageURL.cpp | 3 +++ 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 54f153bc697c..e2781d2823c6 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -22,6 +22,7 @@ namespace DB namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; + extern const int UNSUPPORTED_METHOD; } static const UInt64 max_block_size = 8192; @@ -60,10 +61,9 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) QueryPipeline HTTPDictionarySource::createWrappedBuffer(std::unique_ptr http_buffer_ptr) { - Poco::URI uri(configuration.url); String http_request_compression_method_str = http_buffer_ptr->getCompressionMethod(); auto in_ptr_wrapped - = wrapReadBufferWithCompressionMethod(std::move(http_buffer_ptr), chooseCompressionMethod(uri.getPath(), http_request_compression_method_str)); + = wrapReadBufferWithCompressionMethod(std::move(http_buffer_ptr), chooseCompressionMethod(configuration.url.getPath(), http_request_compression_method_str)); auto source = context->getInputFormat(configuration.format, *in_ptr_wrapped, sample_block, max_block_size); source->addBuffer(std::move(in_ptr_wrapped)); return QueryPipeline(std::move(source)); @@ -90,9 +90,7 @@ QueryPipeline HTTPDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); - Poco::URI uri(configuration.url); - - auto buf = BuilderRWBufferFromHTTP(uri) + auto buf = BuilderRWBufferFromHTTP(configuration.url) .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withSettings(context->getReadSettings()) .withTimeouts(timeouts) @@ -105,7 +103,7 @@ QueryPipeline HTTPDictionarySource::loadAll() QueryPipeline HTTPDictionarySource::loadUpdatedAll() { - Poco::URI uri(configuration.url); + Poco::URI uri = configuration.url; getUpdateFieldAndDate(uri); LOG_TRACE(log, "loadUpdatedAll {}", uri.toString()); @@ -134,9 +132,7 @@ QueryPipeline HTTPDictionarySource::loadIds(const std::vector & ids) out_buffer.finalize(); }; - Poco::URI uri(configuration.url); - - auto buf = BuilderRWBufferFromHTTP(uri) + auto buf = BuilderRWBufferFromHTTP(configuration.url) .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withMethod(Poco::Net::HTTPRequest::HTTP_POST) .withSettings(context->getReadSettings()) @@ -163,9 +159,7 @@ QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const out_buffer.finalize(); }; - Poco::URI uri(configuration.url); - - auto buf = BuilderRWBufferFromHTTP(uri) + auto buf = BuilderRWBufferFromHTTP(configuration.url) .withConnectionGroup(HTTPConnectionGroupType::STORAGE) .withMethod(Poco::Net::HTTPRequest::HTTP_POST) .withSettings(context->getReadSettings()) @@ -200,8 +194,7 @@ DictionarySourcePtr HTTPDictionarySource::clone() const std::string HTTPDictionarySource::toString() const { - Poco::URI uri(configuration.url); - return uri.toString(); + return configuration.url.toString(); } void registerDictionarySourceHTTP(DictionarySourceFactory & factory) @@ -293,13 +286,17 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) auto configuration = HTTPDictionarySource::Configuration { - .url = uri, + .url = Poco::URI(uri), .format = format, .update_field = config.getString(settings_config_prefix + ".update_field", ""), .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), .header_entries = std::move(header_entries) }; + if (configuration.url.getScheme() != "https") { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only https scheme is supported for HTTPDictionarySource"); + } + return std::make_unique(dict_struct, configuration, credentials, sample_block, context); }; factory.registerSource("http", create_table_source); diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index cdf0d9346df9..44fe0c64e960 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -26,7 +26,7 @@ class HTTPDictionarySource final : public IDictionarySource struct Configuration { - const std::string url; + const Poco::URI url; const std::string format; const std::string update_field; const UInt64 update_lag; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index c3cd88dd6014..16f54da63b54 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1528,6 +1528,9 @@ StorageURL::StorageURL( partition_by_, distributed_processing_) { + if (Poco::URI(uri_).getScheme() != "https") { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "URL storage supports only HTTPS protocol"); + } context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); context_->getHTTPHeaderFilter().checkAndNormalizeHeaders(headers); } From 3a1401fe8dec4046ae2366717185b6c5293dfa85 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sat, 13 Dec 2025 17:33:58 +0100 Subject: [PATCH 19/49] Allow avnadmin creating database using sql This commit enables a special user (configured via user_with_indirect_database_creation setting) to create and drop Replicated databases through SQL with automatic parameter setup and privilege management. Main changes: 1. Allow special user calling CREATE DATABASE and DROP DATABASE, which overrides during execution phase, ensuring correct parameters are passed and running it with elevated privileges. 2. Introduce GRANT DEFAULT REPLICATED DATABASE PRIVILEGES statement, which grants a default set of privileges for a database. This maintains a common source of truth while creating databases from different environments. 3. Add server settings: - reserved_replicated_database_prefixes: Prohibits database names with certain prefixes - user_with_indirect_database_creation: User allowed simplified database creation - cluster_database: Reference database for cluster operations 4. Enhance DatabaseReplicated to store shard_macros for reuse when creating new databases. 5. Enforce ON CLUSTER requirement for non-admin users when dropping databases to ensure complete removal across the cluster. Technical details: - Added createReplicatedDatabaseByClient() to handle automatic database creation with proper cluster configuration - Added checkDatabaseNameAllowed() to validate database names - Modified executeDDLQueryOnCluster() to support skipping distributed checks for internal operations - Added setGlobalContext() to Context for executing queries with elevated privileges [DDB-1615] [DDB-1839] [DDB-1968] --- src/Core/ServerSettings.cpp | 3 + src/Databases/DatabaseReplicated.cpp | 7 +- src/Databases/DatabaseReplicated.h | 4 +- .../Access/InterpreterGrantQuery.cpp | 60 +++++++++ src/Interpreters/Context.cpp | 6 + src/Interpreters/Context.h | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 123 ++++++++++++++++-- src/Interpreters/InterpreterCreateQuery.h | 3 + src/Interpreters/InterpreterDropQuery.cpp | 28 +++- src/Interpreters/executeDDLQueryOnCluster.cpp | 7 +- src/Interpreters/executeDDLQueryOnCluster.h | 2 +- src/Parsers/Access/ASTGrantQuery.h | 1 + src/Parsers/Access/ParserGrantQuery.cpp | 42 ++++++ src/Parsers/CommonParsers.h | 1 + 14 files changed, 267 insertions(+), 22 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 004f8a16098c..b1cf1c59e5e2 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1034,6 +1034,9 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ DECLARE(Bool, disable_insertion_and_mutation, false, R"( Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance. )", 0) \ + DECLARE(String, reserved_replicated_database_prefixes, "", R"(Comma separated list of prohibited replicated database prefixes.)", 0) \ + DECLARE(String, user_with_indirect_database_creation, "", R"(Database creation for this user is simplified by setting necessary parameters automatically and prohibiting dangerous behavoir.)", 0) \ + DECLARE(String, cluster_database, "", R"(Database used for cluster creation.)", 0) \ DECLARE(UInt64, parts_kill_delay_period, 30, R"( Period to completely remove parts for SharedMergeTree. Only available in ClickHouse Cloud )", 0) \ diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index b2688388adcf..fe2d796727cd 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -144,12 +144,14 @@ DatabaseReplicated::DatabaseReplicated( UUID uuid, const String & zookeeper_path_, const String & shard_name_, + const String & shard_macros_, const String & replica_name_, DatabaseReplicatedSettings db_settings_, ContextPtr context_) : DatabaseAtomic(name_, metadata_path_, uuid, "DatabaseReplicated (" + name_ + ")", context_) , zookeeper_path(zookeeper_path_) , shard_name(shard_name_) + , shard_macros(shard_macros_) , replica_name(replica_name_) , db_settings(std::move(db_settings_)) , tables_metadata_digest(0) @@ -2340,7 +2342,7 @@ void registerDatabaseReplicated(DatabaseFactory & factory) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.context); String zookeeper_path = safeGetLiteralValue(arguments[0], "Replicated"); - String shard_name = safeGetLiteralValue(arguments[1], "Replicated"); + String shard_macros = safeGetLiteralValue(arguments[1], "Replicated"); String replica_name = safeGetLiteralValue(arguments[2], "Replicated"); /// Expand macros. @@ -2351,7 +2353,7 @@ void registerDatabaseReplicated(DatabaseFactory & factory) info.level = 0; info.table_id.uuid = UUIDHelpers::Nil; - shard_name = args.context->getMacros()->expand(shard_name, info); + String shard_name = args.context->getMacros()->expand(shard_macros, info); info.level = 0; replica_name = args.context->getMacros()->expand(replica_name, info); @@ -2367,6 +2369,7 @@ void registerDatabaseReplicated(DatabaseFactory & factory) args.uuid, zookeeper_path, shard_name, + shard_macros, replica_name, std::move(database_replicated_settings), args.context); }; diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 917d7e0e6c63..b0cb58a08e02 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -46,7 +46,7 @@ class DatabaseReplicated : public DatabaseAtomic static constexpr auto BROKEN_REPLICATED_TABLES_SUFFIX = "_broken_replicated_tables"; DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid, - const String & zookeeper_path_, const String & shard_name_, const String & replica_name_, + const String & zookeeper_path_, const String & shard_name_, const String & shard_macros_, const String & replica_name_, DatabaseReplicatedSettings db_settings_, ContextPtr context); @@ -82,6 +82,7 @@ class DatabaseReplicated : public DatabaseAtomic void stopReplication() override; String getShardName() const { return shard_name; } + String getShardMacros() const { return shard_macros; } String getReplicaName() const { return replica_name; } String getReplicaGroupName() const { return replica_group_name; } String getFullReplicaName() const; @@ -188,6 +189,7 @@ class DatabaseReplicated : public DatabaseAtomic String zookeeper_path; String shard_name; + String shard_macros; String replica_name; String replica_group_name; String replica_path; diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 2b5381e96400..3565930dd802 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -8,11 +9,17 @@ #include #include #include +#include #include #include #include #include #include +#include +#include +#include "Common/escapeString.h" +#include "Databases/IDatabase.h" +#include "base/sleep.h" namespace DB { @@ -20,6 +27,12 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; + extern const int ACCESS_DENIED; +} + +namespace ServerSetting +{ + extern const ServerSettingsString cluster_database; } namespace @@ -470,6 +483,53 @@ BlockIO InterpreterGrantQuery::execute() if (need_check_grantees_are_allowed) current_user_access->checkGranteesAreAllowed(grantees); + if (query.default_replicated_db_privileges) { + auto context = getContext(); + if (query.access_rights_elements.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected number of access rights elements: {}.", query.access_rights_elements.size()); + String db_name = query.access_rights_elements[0].database; + if (query.grantees->names.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected number of grantees."); + String grantee = query.grantees->names[0]; + // We cannot check if database is replicated here because it might not be created yet. + + String cluster_database = context->getServerSettings()[ServerSetting::cluster_database]; + String default_grant_query = "GRANT "; + if (db_name != cluster_database) + default_grant_query += "DROP DATABASE, "; + default_grant_query += + "ALTER UPDATE, " + "ALTER DELETE, " + "ALTER COLUMN, " + "ALTER MODIFY COMMENT, " + "ALTER INDEX, " + "ALTER PROJECTION, " + "ALTER CONSTRAINT, " + "ALTER TTL, " + "ALTER MATERIALIZE TTL, " + "ALTER SETTINGS, " + "ALTER MOVE PARTITION, " + "ALTER FETCH PARTITION, " + "ALTER VIEW, " + // CREATE TABLE implicitly enables CREATE VIEW + "CREATE TABLE, " + // DROP TABLE implicitly enables DROP VIEW + "DROP TABLE, " + "CREATE DICTIONARY, " + "DROP DICTIONARY, " + "dictGet, " + "INSERT, " + "OPTIMIZE, " + "SELECT, " + "SHOW, " + "SYSTEM SYNC REPLICA, " + "TRUNCATE " + "ON " + backQuote(db_name) + ".* TO " + escapeString(grantee) + " WITH GRANT OPTION"; + + auto exec_result = executeQuery(default_grant_query, context, QueryFlags{ .internal = true }); + return {}; + } + AccessRights new_rights; if (query.current_grants) calculateCurrentGrantRightsWithIntersection(new_rights, current_user_access, elements_to_grant); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7f431b59c71b..4ec44f0b66bd 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1765,6 +1765,12 @@ std::shared_ptr Context::getUser() const return getAccess()->getUser(); } +void Context::setGlobalContext() { + std::lock_guard lock(mutex); + user_id = {}; + need_recalculate_access = true; +} + String Context::getUserName() const { return getAccess()->getUserName(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index aaa6a2bc9a64..3d573d62173f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -743,6 +743,8 @@ class Context: public ContextData, public std::enable_shared_from_this void setUser(const UUID & user_id_, const std::vector & external_roles_ = {}); UserPtr getUser() const; + void setGlobalContext(); + std::optional getUserID() const; String getUserName() const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 0e859d775db3..c16ff1902579 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -14,6 +14,8 @@ #include #include #include +#include +#include #include #include #include @@ -32,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -152,10 +155,14 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_table_num_to_throw; extern const ServerSettingsUInt64 max_replicated_table_num_to_throw; extern const ServerSettingsUInt64 max_view_num_to_throw; + extern const ServerSettingsString cluster_database; + extern const ServerSettingsString reserved_replicated_database_prefixes; + extern const ServerSettingsString user_with_indirect_database_creation; } namespace ErrorCodes { + extern const int ACCESS_DENIED; extern const int TABLE_ALREADY_EXISTS; extern const int DICTIONARY_ALREADY_EXISTS; extern const int EMPTY_LIST_OF_COLUMNS_PASSED; @@ -174,11 +181,13 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int ENGINE_REQUIRED; extern const int UNKNOWN_STORAGE; + extern const int SETTING_CONSTRAINT_VIOLATION; extern const int SYNTAX_ERROR; extern const int SUPPORT_IS_DISABLED; extern const int TOO_MANY_TABLES; extern const int TOO_MANY_DATABASES; extern const int THERE_IS_NO_COLUMN; + extern const int UNSUPPORTED_PARAMETER; } namespace fs = std::filesystem; @@ -189,20 +198,7 @@ InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, Contex } -BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) -{ - String database_name = create.getDatabase(); - - auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); - - /// Database can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard - if (DatabaseCatalog::instance().isDatabaseExist(database_name)) - { - if (create.if_not_exists) - return {}; - throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Database {} already exists.", database_name); - } - +void InterpreterCreateQuery::checkMaxDatabaseNumToThrow() { auto db_num_limit = getContext()->getGlobalContext()->getServerSettings()[ServerSetting::max_database_num_to_throw].value; if (db_num_limit > 0 && !internal) { @@ -227,6 +223,24 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "The limit (server configuration parameter `max_database_num_to_throw`) is set to {}, the current number of databases is {}", db_num_limit, db_count); } +} + + +BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) +{ + String database_name = create.getDatabase(); + + auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); + + /// Database can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard + if (DatabaseCatalog::instance().isDatabaseExist(database_name)) + { + if (create.if_not_exists) + return {}; + else + throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Database {} already exists.", database_name); + } + checkMaxDatabaseNumToThrow(); auto default_db_disk = getContext()->getDatabaseDisk(); @@ -2331,6 +2345,67 @@ BlockIO InterpreterCreateQuery::executeQueryOnCluster(ASTCreateQuery & create) return executeDDLQueryOnCluster(query_ptr, getContext(), params); } +BlockIO InterpreterCreateQuery::createReplicatedDatabaseByClient() { + auto & create = query_ptr->as(); + auto context = getContext(); + String cluster_database = context->getServerSettings()[ServerSetting::cluster_database]; + if (cluster_database.empty()) + throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Setting cluster_database should be set."); + auto default_database = DatabaseCatalog::instance().getDatabase(cluster_database); + DatabaseReplicated * replicated_database = dynamic_cast(default_database.get()); + if (!replicated_database) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database default should have Replicated engine"); + String if_not_exists_fragment = ""; + if (create.if_not_exists) + if_not_exists_fragment = " IF NOT EXISTS "; + checkMaxDatabaseNumToThrow(); + String db_name = create.getDatabase(); + String create_db_query = "CREATE DATABASE " + if_not_exists_fragment + backQuote(db_name) + + " ON CLUSTER " + backQuote(cluster_database) + + " ENGINE = Replicated(" + "'/clickhouse/databases/" + escapeForFileName(db_name) + + "', '" + replicated_database->getShardMacros() + "', '{replica}') " + "SETTINGS collection_name='cluster_secret'"; + auto new_context = Context::createCopy(context); + new_context->setGlobalContext(); + new_context->setSetting("allow_distributed_ddl", 1); + executeQuery(create_db_query, new_context, QueryFlags{ .internal = true }); + auto username = context->getUserName(); + String grant_query = "GRANT DEFAULT REPLICATED DATABASE PRIVILEGES ON " + backQuote(db_name) + ".* TO " + escapeString(username); + auto exec_result = executeQuery(grant_query, new_context, QueryFlags{ .internal = true }); + return {}; +} + +void InterpreterCreateQuery::checkDatabaseNameAllowed() { + auto & create = query_ptr->as(); + if (internal) + return; + auto *storage = create.storage; + if (storage && storage->engine && storage->engine->name != "Replicated") + return; + String db_name = create.getDatabase(); + auto context = getContext(); + String prohibited_prefixes = context->getServerSettings()[ServerSetting::reserved_replicated_database_prefixes]; + if (prohibited_prefixes.empty()) + return; + Tokens tokens(prohibited_prefixes.data(), prohibited_prefixes.data() + prohibited_prefixes.size()); + IParser::Pos pos(tokens, 1, 1); + Expected expected; + + /// Use an unordered list rather than string vector + auto check_name_allowed = [&] + { + String prefix; + if (!parseIdentifierOrStringLiteral(pos, expected, prefix)) + throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Cannot parse reserved_replicated_database_prefixes setting."); + if (db_name.starts_with(prefix)) + throw Exception(ErrorCodes::ACCESS_DENIED, "Database name cannot start with '{}'", prefix); + return true; + }; + ParserList::parseUtil(pos, expected, check_name_allowed, false); +} + + BlockIO InterpreterCreateQuery::execute() { FunctionNameNormalizer::visit(query_ptr.get()); @@ -2339,6 +2414,26 @@ BlockIO InterpreterCreateQuery::execute() create.if_not_exists |= getContext()->getSettingsRef()[Setting::create_if_not_exists]; bool is_create_database = create.database && !create.table; + if (is_create_database) + checkDatabaseNameAllowed(); + auto context = getContext(); + auto username = context->getUserName(); + String user_with_interect_db_creation = context->getServerSettings()[ServerSetting::user_with_indirect_database_creation]; + if (is_create_database && !user_with_interect_db_creation.empty() && username == user_with_interect_db_creation && !internal) { + auto *storage = create.storage; + if (storage) { + if (storage->engine && storage->engine->name != "Replicated") + throw Exception(ErrorCodes::ACCESS_DENIED, "Only Replicated database can be created through SQL."); + if (storage->engine && storage->engine->arguments && storage->engine->arguments->children.size() > 0) + throw Exception(ErrorCodes::UNSUPPORTED_PARAMETER, "Arguments cannot be specified for Replicated database engine."); + if (storage->settings && storage->settings->changes.size() > 0) { + throw Exception(ErrorCodes::UNSUPPORTED_PARAMETER, "Settings are not allowed for Replicated database."); + } + } + if (!create.cluster.empty()) + throw Exception(ErrorCodes::UNSUPPORTED_PARAMETER, "ON CLUSTER cannot be used in CREATE DATABASE, it will be set implicitly."); + return createReplicatedDatabaseByClient(); + } if (!create.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) { if (create.attach_as_replicated.has_value()) diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 65e7ac5962a2..8082b564935c 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -97,6 +97,9 @@ class InterpreterCreateQuery : public IInterpreter, WithMutableContext bool columns_inferred_from_select_query = false; }; + BlockIO createReplicatedDatabaseByClient(); + void checkDatabaseNameAllowed(); + void checkMaxDatabaseNumToThrow(); BlockIO createDatabase(ASTCreateQuery & create); BlockIO createTable(ASTCreateQuery & create); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 4e2dd6cb83a6..e7522d9ba358 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -20,6 +21,7 @@ #include #include #include +#include #include #include "config.h" @@ -39,8 +41,15 @@ namespace Setting extern const SettingsSeconds lock_acquire_timeout; } +namespace ServerSetting +{ + extern const ServerSettingsString cluster_database; +} + namespace ErrorCodes { + extern const int ACCESS_DENIED; + extern const int SETTING_CONSTRAINT_VIOLATION; extern const int LOGICAL_ERROR; extern const int SYNTAX_ERROR; extern const int UNKNOWN_TABLE; @@ -91,13 +100,30 @@ BlockIO InterpreterDropQuery::executeSingleDropQuery(const ASTPtr & drop_query_p if (getContext()->getSettingsRef()[Setting::database_atomic_wait_for_drop_and_detach_synchronously]) drop.sync = true; + // Enforce `ON CLUSTER {default}` for ordinary users to ensure complete database removal. + auto query_context = getContext(); + auto access = query_context->getAccess(); + auto is_drop_database = drop.database && !drop.table; + if (is_drop_database + && !maybeRemoveOnCluster(current_query_ptr, getContext()) + && !access->isGranted(AccessType::ACCESS_MANAGEMENT)) { + if (drop.kind == ASTDropQuery::Kind::Detach) + throw Exception(ErrorCodes::ACCESS_DENIED, "Database detach is not allowed."); + String cluster_database = query_context->getServerSettings()[ServerSetting::cluster_database]; + if (cluster_database.empty()) + throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Setting cluster_database should be set."); + if (!drop.cluster.empty() && drop.cluster != cluster_database) + throw Exception(ErrorCodes::ACCESS_DENIED, "Cannot execute query on specified cluster."); + drop.cluster = cluster_database; + } + if (drop.table) return executeToTable(drop); if (drop.database && !drop.cluster.empty() && !maybeRemoveOnCluster(current_query_ptr, getContext())) { DDLQueryOnClusterParams params; params.access_to_check = getRequiredAccessForDDLOnCluster(); - return executeDDLQueryOnCluster(current_query_ptr, getContext(), params); + return executeDDLQueryOnCluster(current_query_ptr, query_context, params, true); } if (drop.database) return executeToDatabase(drop); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 46ef1aaafee3..92f85add4ae6 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -61,7 +61,7 @@ bool isSupportedAlterTypeForOnClusterDDLQuery(int type) } -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, const DDLQueryOnClusterParams & params) +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, const DDLQueryOnClusterParams & params, bool skip_distributed_checks) { OpenTelemetry::SpanHolder span(__FUNCTION__, OpenTelemetry::SpanKind::PRODUCER); @@ -79,7 +79,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Distributed execution is not supported for such DDL queries"); } - if (!context->getSettingsRef()[Setting::allow_distributed_ddl]) + if (!skip_distributed_checks && !context->getSettingsRef()[Setting::allow_distributed_ddl]) throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Distributed DDL queries are prohibited for the user"); bool is_system_query = dynamic_cast(query_ptr.get()) != nullptr; @@ -110,7 +110,8 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Distributed DDL queries are prohibited for the cluster"); /// TODO: support per-cluster grant - context->checkAccess(AccessType::CLUSTER); + if (!skip_distributed_checks) + context->checkAccess(AccessType::CLUSTER); /// NOTE: if `async_load_databases = true`, then it block until ddl_worker is started, which includes startup of all related tables. DDLWorker & ddl_worker = context->getDDLWorker(); diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 69e0c38834e6..3f35278ea290 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -44,7 +44,7 @@ struct DDLQueryOnClusterParams /// Pushes distributed DDL query to the queue. /// Returns DDLQueryStatusSource, which reads results of query execution on each host in the cluster. -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const DDLQueryOnClusterParams & params = {}); +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const DDLQueryOnClusterParams & params = {}, bool skip_distributed_checks = false); BlockIO getDDLOnClusterStatus(const String & node_path, const String & replicas_path, const DDLLogEntry & entry, ContextPtr context); diff --git a/src/Parsers/Access/ASTGrantQuery.h b/src/Parsers/Access/ASTGrantQuery.h index b290807456bd..3dcd2bfbed52 100644 --- a/src/Parsers/Access/ASTGrantQuery.h +++ b/src/Parsers/Access/ASTGrantQuery.h @@ -27,6 +27,7 @@ class ASTGrantQuery : public IAST, public ASTQueryWithOnCluster bool replace_access = false; bool replace_granted_roles = false; bool current_grants = false; + bool default_replicated_db_privileges = false; std::shared_ptr grantees; diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index f928550cba92..6782efbf7dd4 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; + extern const int NOT_IMPLEMENTED; } namespace @@ -53,6 +54,30 @@ namespace return true; } + bool parseDefaultPrivileges(IParser::Pos & pos, Expected & expected, AccessRightsElements & elements) + { + AccessRightsElement default_element(AccessType::ALL); + + if (!ParserKeyword{Keyword::ON}.ignore(pos, expected)) + return false; + + String database_name; + String table_name; + bool wildcard = false; + bool default_database = false; + if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, table_name, wildcard, default_database)) + return false; + + if (database_name.empty() || !table_name.empty() || wildcard) + throw Exception(ErrorCodes::SYNTAX_ERROR, "DEFAULT PRIVILEGES can only be applied to all tables inside a database"); + + default_element.database = database_name; + default_element.table = table_name; + default_element.wildcard = wildcard; + default_element.default_database = default_database; + elements.push_back(std::move(default_element)); + return true; + } bool parseRoles(IParser::Pos & pos, Expected & expected, bool is_revoke, bool id_mode, std::shared_ptr & roles) { @@ -130,12 +155,19 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) std::shared_ptr roles; bool current_grants = false; + bool default_replicated_db_grants = false; if (!is_revoke && ParserKeyword{Keyword::CURRENT_GRANTS}.ignore(pos, expected)) { current_grants = true; if (!parseCurrentGrants(pos, expected, elements)) return false; } + else if (!is_revoke && ParserKeyword{Keyword::DEFAULT_REPLICATED_DATABASE_PRIVILEGES}.ignore(pos, expected)) + { + if (!parseDefaultPrivileges(pos, expected, elements)) + return false; + default_replicated_db_grants = true; + } else { if (!parseAccessRightsElementsWithoutOptions(pos, expected, elements) && !parseRoles(pos, expected, is_revoke, attach_mode, roles)) @@ -148,6 +180,12 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) std::shared_ptr grantees; if (!parseToGrantees(pos, expected, is_revoke, grantees) && !allow_no_grantees) return false; + if (default_replicated_db_grants) { + if (grantees->names.size() != 1) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEFAULT REPLICATED DATABASE PRIVILEGES can only be applied to one grantee"); + if (grantees->current_user || grantees->all || grantees->except_current_user || grantees->except_names.size() > 0) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DEFAULT REPLICATED DATABASE PRIVILEGES should be applied to a concrete user or role"); + } if (cluster.empty()) parseOnCluster(pos, expected, cluster); @@ -166,6 +204,9 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (cluster.empty()) parseOnCluster(pos, expected, cluster); + if (default_replicated_db_grants && !cluster.empty()) + throw Exception(ErrorCodes::SYNTAX_ERROR, "DEFAULT REPLICATED DATABASE PRIVILEGES cannot be executed on cluster."); + if (grant_option && roles) throw Exception(ErrorCodes::SYNTAX_ERROR, "GRANT OPTION should be specified for access types"); if (admin_option && !elements.empty()) @@ -204,6 +245,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->replace_access = replace_access; query->replace_granted_roles = replace_role; query->current_grants = current_grants; + query->default_replicated_db_privileges = default_replicated_db_grants; return true; } diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 057aad6fffea..4ff75fe57407 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -136,6 +136,7 @@ namespace DB MR_MACROS(DEALLOCATE, "DEALLOCATE") \ MR_MACROS(DEDUPLICATE, "DEDUPLICATE") \ MR_MACROS(DEFAULT_DATABASE, "DEFAULT DATABASE") \ + MR_MACROS(DEFAULT_REPLICATED_DATABASE_PRIVILEGES, "DEFAULT REPLICATED DATABASE PRIVILEGES") \ MR_MACROS(DEFAULT_ROLE, "DEFAULT ROLE") \ MR_MACROS(DEFAULT, "DEFAULT") \ MR_MACROS(DEFINER, "DEFINER") \ From 3107e730cf3483db545ed783cd5aff1e37f41c25 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Sat, 13 Dec 2025 18:50:13 +0100 Subject: [PATCH 20/49] Add CHECK TABLE to default privileges CHECK TABLE requires an explicit grant since e96e0ae. This commit adds CHECK to the default privileges granted when using GRANT DEFAULT REPLICATED DATABASE PRIVILEGES, ensuring users can run CHECK TABLE on tables in replicated databases without requiring an additional grant. The change adds "CHECK, " to the privilege list in InterpreterGrantQuery::execute() when handling default replicated database privileges. Co-authored-by: Aliaksei Khatskevich --- src/Interpreters/Access/InterpreterGrantQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 3565930dd802..0ea3fa0aa82c 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -522,6 +522,7 @@ BlockIO InterpreterGrantQuery::execute() "OPTIMIZE, " "SELECT, " "SHOW, " + "CHECK, " "SYSTEM SYNC REPLICA, " "TRUNCATE " "ON " + backQuote(db_name) + ".* TO " + escapeString(grantee) + " WITH GRANT OPTION"; From 3132b13d0264a26c98335a68bf79de5fa537a67b Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 15 Dec 2025 10:14:24 +0100 Subject: [PATCH 21/49] Add SSL/TLS configuration support for PostgreSQL and MySQL connections This commit adds comprehensive SSL/TLS configuration capabilities for PostgreSQL and MySQL database connections in ClickHouse, along with a security fix for the MariaDB connector. Changes: 1. MariaDB Connector/C Security Fix: - Updated submodule to aiven/mariadb-connector-c fork - Fixed X509_check_host call to include hostname length parameter - Prevents potential certificate validation bypass vulnerabilities 2. PostgreSQL SSL Configuration: - Added SSLMode enum (DISABLE, ALLOW, PREFER, REQUIRE, VERIFY_CA, VERIFY_FULL) - Added server settings: * postgresql_connection_pool_ssl_mode (default: PREFER) * postgresql_connection_pool_ssl_root_cert (default: empty) - Updated PoolWithFailover to accept SSL mode and CA certificate path - Modified formatConnectionString to include sslmode and sslrootcert parameters - Integrated SSL settings across all PostgreSQL integration points: * DatabasePostgreSQL * DatabaseMaterializedPostgreSQL * StoragePostgreSQL * StorageMaterializedPostgreSQL * TableFunctionPostgreSQL * PostgreSQLDictionarySource 3. MySQL SSL Configuration: - Added MySQLSSLMode enum (DISABLE, PREFER, VERIFY_FULL) - Updated Connection, Pool, and PoolWithFailover classes to accept SSL mode - Added ssl_mode and ssl_root_cert to StorageMySQL::Configuration - Enhanced MySQL dictionary source to support ssl_mode in named collections - Integrated SSL settings in MySQLHelpers and StorageMySQL Security Benefits: - Enables encrypted connections to prevent data interception - Supports certificate validation to prevent man-in-the-middle attacks - Provides flexible SSL mode selection for different security requirements - Fixes critical certificate hostname validation bug in MariaDB connector The changes maintain backward compatibility with default SSL mode set to PREFER, which attempts SSL but falls back gracefully if unavailable. Co-authored-by: Joe Lynch --- .gitmodules | 3 +- contrib/mariadb-connector-c | 2 +- src/Common/mysqlxx/Connection.cpp | 16 ++++++++-- src/Common/mysqlxx/Pool.cpp | 4 +++ src/Common/mysqlxx/PoolWithFailover.cpp | 4 ++- src/Common/mysqlxx/mysqlxx/Connection.h | 4 +++ src/Common/mysqlxx/mysqlxx/Pool.h | 5 ++- src/Common/mysqlxx/mysqlxx/PoolWithFailover.h | 2 ++ src/Core/PostgreSQL/PoolWithFailover.cpp | 32 +++++++++++++++++-- src/Core/PostgreSQL/PoolWithFailover.h | 9 ++++++ src/Core/PostgreSQL/Utils.cpp | 15 ++++++++- src/Core/PostgreSQL/Utils.h | 15 +++++++-- src/Core/Settings.cpp | 6 ++++ src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 13 ++++++++ src/Core/SettingsEnums.h | 21 ++++++++++++ .../DatabaseMaterializedPostgreSQL.cpp | 5 ++- .../PostgreSQL/DatabasePostgreSQL.cpp | 6 +++- src/Dictionaries/MySQLDictionarySource.cpp | 18 +++-------- .../PostgreSQLDictionarySource.cpp | 4 +++ src/Storages/MySQL/MySQLHelpers.cpp | 7 ++-- src/Storages/MySQL/MySQLHelpers.h | 1 + .../StorageMaterializedPostgreSQL.cpp | 5 ++- src/Storages/StorageMySQL.cpp | 5 ++- src/Storages/StorageMySQL.h | 3 ++ src/Storages/StoragePostgreSQL.cpp | 13 ++++++-- src/Storages/StoragePostgreSQL.h | 3 ++ .../TableFunctionPostgreSQL.cpp | 6 +++- 28 files changed, 193 insertions(+), 35 deletions(-) diff --git a/.gitmodules b/.gitmodules index bc363d734f90..4fa083b72211 100644 --- a/.gitmodules +++ b/.gitmodules @@ -30,7 +30,8 @@ url = https://github.com/google/re2 [submodule "contrib/mariadb-connector-c"] path = contrib/mariadb-connector-c - url = https://github.com/ClickHouse/mariadb-connector-c + url = https://github.com/aiven/mariadb-connector-c + branch = aiven/clickhouse-v25.8.12.129 [submodule "contrib/jemalloc"] path = contrib/jemalloc url = https://github.com/jemalloc/jemalloc diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c index d0a788c5b9fc..107011a3e98f 160000 --- a/contrib/mariadb-connector-c +++ b/contrib/mariadb-connector-c @@ -1 +1 @@ -Subproject commit d0a788c5b9fcaca2368d9233770d3ca91ea79f88 +Subproject commit 107011a3e98fc118476c14ada1ab95178e18645f diff --git a/src/Common/mysqlxx/Connection.cpp b/src/Common/mysqlxx/Connection.cpp index 77d1dcd189fd..3a1a7a33af5f 100644 --- a/src/Common/mysqlxx/Connection.cpp +++ b/src/Common/mysqlxx/Connection.cpp @@ -4,6 +4,7 @@ #include #endif +#include #include #include @@ -49,13 +50,14 @@ Connection::Connection( const char* ssl_ca, const char* ssl_cert, const char* ssl_key, + DB::MySQLSSLMode ssl_mode, unsigned timeout, unsigned rw_timeout, bool enable_local_infile, bool opt_reconnect) : Connection() { - connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout, enable_local_infile, opt_reconnect); + connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, ssl_mode, timeout, rw_timeout, enable_local_infile, opt_reconnect); } Connection::Connection(const std::string & config_name) @@ -79,6 +81,7 @@ void Connection::connect(const char* db, const char * ssl_ca, const char * ssl_cert, const char * ssl_key, + DB::MySQLSSLMode ssl_mode, unsigned timeout, unsigned rw_timeout, bool enable_local_infile, @@ -111,8 +114,15 @@ void Connection::connect(const char* db, throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); /// Specifies particular ssl key and certificate if it needs - if (mysql_ssl_set(driver.get(), ifNotEmpty(ssl_key), ifNotEmpty(ssl_cert), ifNotEmpty(ssl_ca), nullptr, nullptr)) - throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); + if (ssl_mode != DB::MySQLSSLMode::DISABLE) { + if (mysql_ssl_set(driver.get(), ifNotEmpty(ssl_key), ifNotEmpty(ssl_cert), ifNotEmpty(ssl_ca), nullptr, nullptr)) + throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); + } + if (ssl_mode == DB::MySQLSSLMode::VERIFY_FULL) { + static const char enable = 1; + if (mysql_options(driver.get(), MYSQL_OPT_SSL_VERIFY_SERVER_CERT, &enable)) + throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); + } if (!mysql_real_connect(driver.get(), server, user, password, db, port, ifNotEmpty(socket), driver->client_flag)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index 7173adf3326c..2e8657e087da 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -135,6 +135,7 @@ Pool::Pool( const std::string & ssl_ca_, const std::string & ssl_cert_, const std::string & ssl_key_, + DB::MySQLSSLMode ssl_mode_, const std::string & socket_, unsigned connect_timeout_, unsigned rw_timeout_, @@ -155,6 +156,7 @@ Pool::Pool( , ssl_ca(ssl_ca_) , ssl_cert(ssl_cert_) , ssl_key(ssl_key_) + , ssl_mode(ssl_mode_) , enable_local_infile(enable_local_infile_) , opt_reconnect(opt_reconnect_) { @@ -311,6 +313,7 @@ void Pool::Entry::forceConnected() const pool->ssl_ca.c_str(), pool->ssl_cert.c_str(), pool->ssl_key.c_str(), + pool->ssl_mode, pool->connect_timeout, pool->rw_timeout, pool->enable_local_infile, @@ -387,6 +390,7 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) ssl_ca.c_str(), ssl_cert.c_str(), ssl_key.c_str(), + ssl_mode, connect_timeout, rw_timeout, enable_local_infile, diff --git a/src/Common/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp index 31f4ffc8adf1..f1b571e5042c 100644 --- a/src/Common/mysqlxx/PoolWithFailover.cpp +++ b/src/Common/mysqlxx/PoolWithFailover.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -124,6 +125,7 @@ PoolWithFailover::PoolWithFailover( const std::string & ssl_ca, const std::string & ssl_cert, const std::string & ssl_key, + DB::MySQLSSLMode ssl_mode, unsigned default_connections_, unsigned max_connections_, size_t max_tries_, @@ -140,7 +142,7 @@ PoolWithFailover::PoolWithFailover( for (const auto & [host, port] : addresses) { replicas_by_priority[0].emplace_back(std::make_shared(database, - host, user, password, port, ssl_ca, ssl_cert, ssl_key, + host, user, password, port, ssl_ca, ssl_cert, ssl_key, ssl_mode, /* socket_ = */ "", connect_timeout_, rw_timeout_, diff --git a/src/Common/mysqlxx/mysqlxx/Connection.h b/src/Common/mysqlxx/mysqlxx/Connection.h index fd4b3502e74a..ab9c2bdc97ec 100644 --- a/src/Common/mysqlxx/mysqlxx/Connection.h +++ b/src/Common/mysqlxx/mysqlxx/Connection.h @@ -5,6 +5,7 @@ #include +#include #include #include @@ -78,6 +79,7 @@ class Connection final : private boost::noncopyable const char * ssl_ca = "", const char * ssl_cert = "", const char * ssl_key = "", + DB::MySQLSSLMode ssl_mode = DB::MySQLSSLMode::PREFER, unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE, @@ -99,6 +101,7 @@ class Connection final : private boost::noncopyable const char* ssl_ca, const char* ssl_cert, const char* ssl_key, + DB::MySQLSSLMode ssl_mode = DB::MySQLSSLMode::PREFER, unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE, @@ -140,6 +143,7 @@ class Connection final : private boost::noncopyable ssl_ca.c_str(), ssl_cert.c_str(), ssl_key.c_str(), + DB::MySQLSSLMode::PREFER, timeout, rw_timeout, enable_local_infile, diff --git a/src/Common/mysqlxx/mysqlxx/Pool.h b/src/Common/mysqlxx/mysqlxx/Pool.h index 8382e36448de..7806f798ffcc 100644 --- a/src/Common/mysqlxx/mysqlxx/Pool.h +++ b/src/Common/mysqlxx/mysqlxx/Pool.h @@ -10,6 +10,7 @@ #include #include +#include #include @@ -158,6 +159,7 @@ class Pool final const std::string & ssl_ca_ = "", const std::string & ssl_cert_ = "", const std::string & ssl_key_ = "", + DB::MySQLSSLMode ssl_mode_ = DB::MySQLSSLMode::PREFER, const std::string & socket_ = "", unsigned connect_timeout_ = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout_ = MYSQLXX_DEFAULT_RW_TIMEOUT, @@ -173,7 +175,7 @@ class Pool final user{other.user}, password{other.password}, port{other.port}, socket{other.socket}, connect_timeout{other.connect_timeout}, rw_timeout{other.rw_timeout}, - ssl_ca(other.ssl_ca), ssl_cert(other.ssl_cert), ssl_key(other.ssl_key), + ssl_ca(other.ssl_ca), ssl_cert(other.ssl_cert), ssl_key(other.ssl_key), ssl_mode{other.ssl_mode}, enable_local_infile{other.enable_local_infile}, opt_reconnect(other.opt_reconnect) {} @@ -235,6 +237,7 @@ class Pool final std::string ssl_ca; std::string ssl_cert; std::string ssl_key; + DB::MySQLSSLMode ssl_mode; bool enable_local_infile; bool opt_reconnect; diff --git a/src/Common/mysqlxx/mysqlxx/PoolWithFailover.h b/src/Common/mysqlxx/mysqlxx/PoolWithFailover.h index bf4abaee47c2..f0e7e998cde8 100644 --- a/src/Common/mysqlxx/mysqlxx/PoolWithFailover.h +++ b/src/Common/mysqlxx/mysqlxx/PoolWithFailover.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -126,6 +127,7 @@ namespace mysqlxx const std::string & ssl_ca, const std::string & ssl_cert, const std::string & ssl_key, + DB::MySQLSSLMode ssl_mode, unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index 9498a798a2ee..d1360e2c6749 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -21,6 +21,24 @@ namespace ErrorCodes } } +static std::pair +get_ssl_context(const DB::StoragePostgreSQL::Configuration & configuration, const DB::SSLMode & ssl_mode_, const String & ssl_root_cert_) +{ + DB::SSLMode ssl_mode; + String ssl_root_cert; + if (configuration.ssl_mode) + { + ssl_mode = configuration.ssl_mode.value(); + ssl_root_cert = configuration.ssl_root_cert; + } + else + { + ssl_mode = ssl_mode_; + ssl_root_cert = ssl_root_cert_; + } + return {ssl_mode, ssl_root_cert}; +} + namespace postgres { @@ -89,6 +107,8 @@ PoolWithFailover::PoolWithFailover( size_t max_tries_, bool auto_close_connection_, size_t connection_attempt_timeout_, + const SSLMode & ssl_mode_, + const String & ssl_root_cert_, bool bg_reconnect_) : pool_wait_timeout(pool_wait_timeout_) , max_tries(max_tries_) @@ -102,13 +122,16 @@ PoolWithFailover::PoolWithFailover( { for (const auto & replica_configuration : configurations) { + const auto& [ssl_mode, ssl_root_cert] = get_ssl_context(replica_configuration, ssl_mode_, ssl_root_cert_); auto connection_info = formatConnectionString( replica_configuration.database, replica_configuration.host, replica_configuration.port, replica_configuration.username, replica_configuration.password, - connection_attempt_timeout_); + connection_attempt_timeout_, + ssl_mode, + ssl_root_cert); replicas_with_priority[priority].emplace_back(std::make_shared(connection_info, pool_size)); if (bg_reconnect) DB::ReplicasReconnector::instance().add(connectionReestablisher(std::weak_ptr(replicas_with_priority[priority].back()), pool_wait_timeout)); @@ -123,6 +146,8 @@ PoolWithFailover::PoolWithFailover( size_t max_tries_, bool auto_close_connection_, size_t connection_attempt_timeout_, + const SSLMode & ssl_mode_, + const String & ssl_root_cert_, bool bg_reconnect_) : pool_wait_timeout(pool_wait_timeout_) , max_tries(max_tries_) @@ -136,13 +161,16 @@ PoolWithFailover::PoolWithFailover( for (const auto & [host, port] : configuration.addresses) { LOG_DEBUG(getLogger("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port); + const auto& [ssl_mode, ssl_root_cert] = get_ssl_context(configuration, ssl_mode_, ssl_root_cert_); auto connection_string = formatConnectionString( configuration.database, host, port, configuration.username, configuration.password, - connection_attempt_timeout_); + connection_attempt_timeout_, + ssl_mode, + ssl_root_cert); replicas_with_priority[0].emplace_back(std::make_shared(connection_string, pool_size)); if (bg_reconnect) DB::ReplicasReconnector::instance().add(connectionReestablisher(std::weak_ptr(replicas_with_priority[0].back()), pool_wait_timeout)); diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index 7e42105b3086..2ae0b3f45556 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -8,15 +8,20 @@ #include #include +#include #include #include static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16; static constexpr inline auto POSTGRESQL_POOL_WAIT_TIMEOUT = 5000; +static constexpr inline auto POSTGRESQL_POOL_DEFAULT_CONNECT_TIMEOUT_SEC = 10; +static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SSL_MODE = DB::SSLMode::PREFER; +static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SSL_ROOT_CERT = ""; namespace postgres { + using SSLMode = DB::SSLMode; class PoolWithFailover { @@ -31,6 +36,8 @@ class PoolWithFailover size_t max_tries_, bool auto_close_connection_, size_t connection_attempt_timeout_, + const SSLMode & ssl_mode_, + const String & ssl_root_cert_, bool bg_reconnect_ = false); explicit PoolWithFailover( @@ -40,6 +47,8 @@ class PoolWithFailover size_t max_tries_, bool auto_close_connection_, size_t connection_attempt_timeout_, + const SSLMode & ssl_mode_, + const String & ssl_root_cert_, bool bg_reconnect_ = false); PoolWithFailover(const PoolWithFailover & other) = delete; diff --git a/src/Core/PostgreSQL/Utils.cpp b/src/Core/PostgreSQL/Utils.cpp index 80175d4acdf2..0712afedda4b 100644 --- a/src/Core/PostgreSQL/Utils.cpp +++ b/src/Core/PostgreSQL/Utils.cpp @@ -8,7 +8,15 @@ namespace postgres { -ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password, UInt64 timeout) +ConnectionInfo formatConnectionString( + String dbname, + String host, + UInt16 port, + String user, + String password, + UInt64 timeout, + std::optional ssl_mode, + String ssl_root_cert) { DB::WriteBufferFromOwnString out; out << "dbname=" << DB::quote << dbname @@ -16,7 +24,12 @@ ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, S << " port=" << port << " user=" << DB::quote << user << " password=" << DB::quote << password + << " application_name=clickhouse" << " connect_timeout=" << timeout; + if(ssl_mode) + out << " sslmode=" << DB::SettingFieldSSLMode(ssl_mode.value()).toString(); + if (!ssl_root_cert.empty()) + out << " sslrootcert=" << DB::quote << ssl_root_cert; return {out.str(), host + ':' + DB::toString(port)}; } diff --git a/src/Core/PostgreSQL/Utils.h b/src/Core/PostgreSQL/Utils.h index 04603e678ed3..e0f0ff136c43 100644 --- a/src/Core/PostgreSQL/Utils.h +++ b/src/Core/PostgreSQL/Utils.h @@ -7,6 +7,8 @@ #include #include #include +#include +#include "PoolWithFailover.h" namespace pqxx { @@ -16,8 +18,17 @@ namespace pqxx namespace postgres { - -ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password, UInt64 timeout); +using SSLMode = DB::SSLMode; + +ConnectionInfo formatConnectionString( + String dbname, + String host, + UInt16 port, + String user, + String password, + UInt64 timeout = POSTGRESQL_POOL_DEFAULT_CONNECT_TIMEOUT_SEC, + std::optional ssl_mode = POSTGRESQL_POOL_DEFAULT_SSL_MODE, + String ssl_root_cert = POSTGRESQL_POOL_DEFAULT_SSL_ROOT_CERT); String getConnectionForLog(const String & host, UInt16 port); diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 45f907c63c8e..8aec7c12ce85 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -3851,6 +3851,12 @@ Connection pool push/pop retries number for PostgreSQL table engine and database )", 0) \ DECLARE(Bool, postgresql_connection_pool_auto_close_connection, false, R"( Close connection before returning connection to the pool. +)", 0) \ + DECLARE(SSLMode, postgresql_connection_pool_ssl_mode, SSLMode::PREFER, R"( +Connection pool SSL mode when connecting to source server. +)", 0) \ + DECLARE(String, postgresql_connection_pool_ssl_root_cert, "", R"( +Connection pool SSL root certificate to authenticate the source server when using verify-ca or verify-full. Will use ~/.postgresql/root.crt if empty. )", 0) \ DECLARE(Float, postgresql_fault_injection_probability, 0.0f, R"( Approximate probability of failing internal (for replication) PostgreSQL queries. Valid value is in interval [0.0f, 1.0f] diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f284dd78b4e..85c48527c89e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -105,6 +105,7 @@ class WriteBuffer; M(CLASS_NAME, TransactionsWaitCSNMode) \ M(CLASS_NAME, UInt64) \ M(CLASS_NAME, UInt64Auto) \ + M(CLASS_NAME, SSLMode) \ M(CLASS_NAME, URI) \ M(CLASS_NAME, VectorSearchFilterStrategy) \ M(CLASS_NAME, GeoToH3ArgumentOrder) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 7a586f51168e..647828fee91e 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -106,6 +106,19 @@ IMPLEMENT_SETTING_ENUM(DateTimeInputFormat, ErrorCodes::BAD_ARGUMENTS, {"best_effort", FormatSettings::DateTimeInputFormat::BestEffort}, {"best_effort_us", FormatSettings::DateTimeInputFormat::BestEffortUS}}) +IMPLEMENT_SETTING_ENUM(SSLMode,ErrorCodes::BAD_ARGUMENTS, + {{"disable", SSLMode::DISABLE}, + {"allow", SSLMode::ALLOW}, + {"prefer", SSLMode::PREFER}, + {"require", SSLMode::REQUIRE}, + {"verify-ca", SSLMode::VERIFY_CA}, + {"verify-full", SSLMode::VERIFY_FULL}}) + +IMPLEMENT_SETTING_ENUM(MySQLSSLMode,ErrorCodes::BAD_ARGUMENTS, + {{"disable", MySQLSSLMode::DISABLE}, + {"prefer", MySQLSSLMode::PREFER}, + {"verify-full", MySQLSSLMode::VERIFY_FULL}}) + IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {{"simple", FormatSettings::DateTimeOutputFormat::Simple}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 18873a0790ae..77b8b14737d3 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -289,6 +289,27 @@ DECLARE_SETTING_ENUM_WITH_RENAME(ArrowCompression, FormatSettings::ArrowCompress DECLARE_SETTING_ENUM_WITH_RENAME(ORCCompression, FormatSettings::ORCCompression) +enum class SSLMode +{ + DISABLE = 0, + ALLOW = 1, + PREFER = 2, + REQUIRE = 3, + VERIFY_CA = 4, + VERIFY_FULL = 5, +}; + +DECLARE_SETTING_ENUM(SSLMode) + +// Only this subset of SSL modes are supported for MySQL +enum class MySQLSSLMode { + DISABLE = 0, + PREFER = 2, + VERIFY_FULL = 5, +}; + +DECLARE_SETTING_ENUM(MySQLSSLMode) + enum class Dialect : uint8_t { clickhouse, diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index bd21af241fb2..5e969fd44057 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -547,7 +548,9 @@ void registerDatabaseMaterializedPostgreSQL(DatabaseFactory & factory) configuration.port, configuration.username, configuration.password, - args.context->getSettingsRef()[Setting::postgresql_connection_attempt_timeout]); + args.context->getSettingsRef()[Setting::postgresql_connection_attempt_timeout], + configuration.ssl_mode.value_or(SSLMode::PREFER), + configuration.ssl_root_cert); auto postgresql_replica_settings = std::make_unique(); if (engine_define->settings) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 424926ec86d9..e2ffc46fcf6e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -42,6 +42,8 @@ namespace Setting extern const SettingsUInt64 postgresql_connection_pool_retries; extern const SettingsBool postgresql_connection_pool_auto_close_connection; extern const SettingsUInt64 postgresql_connection_attempt_timeout; + extern const SettingsSSLMode postgresql_connection_pool_ssl_mode; + extern const SettingsString postgresql_connection_pool_ssl_root_cert; } namespace ErrorCodes @@ -591,7 +593,9 @@ void registerDatabasePostgreSQL(DatabaseFactory & factory) settings[Setting::postgresql_connection_pool_wait_timeout], settings[Setting::postgresql_connection_pool_retries], settings[Setting::postgresql_connection_pool_auto_close_connection], - settings[Setting::postgresql_connection_attempt_timeout]); + settings[Setting::postgresql_connection_attempt_timeout], + static_cast(settings[Setting::postgresql_connection_pool_ssl_mode]), + static_cast(settings[Setting::postgresql_connection_pool_ssl_root_cert])); return std::make_shared( args.context, diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 7ba8eb01c6f6..a8524548f650 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -60,7 +61,7 @@ static const ValidateKeysMultiset dictionary_allow "dont_check_update_time" /* obsolete */, "query", "where", "name" /* name_collection */, "socket", "share_connection", "fail_on_connection_loss", "close_connection", - "ssl_ca", "ssl_cert", "ssl_key", + "ssl_ca", "ssl_cert", "ssl_key", "ssl_mode", "enable_local_infile", "opt_reconnect", "connect_timeout", "mysql_connect_timeout", "mysql_rw_timeout", "rw_timeout"}; @@ -141,23 +142,12 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) named_collection->getOrDefault("ssl_ca", ""), named_collection->getOrDefault("ssl_cert", ""), named_collection->getOrDefault("ssl_key", ""), + SettingFieldMySQLSSLModeTraits::fromString(named_collection->getOrDefault("ssl_mode", "prefer")), mysql_settings)); } else { - dictionary_configuration.emplace(MySQLDictionarySource::Configuration{ - .db = config.getString(settings_config_prefix + ".db", ""), - .table = config.getString(settings_config_prefix + ".table", ""), - .query = config.getString(settings_config_prefix + ".query", ""), - .where = config.getString(settings_config_prefix + ".where", ""), - .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), - .update_field = config.getString(settings_config_prefix + ".update_field", ""), - .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), - .bg_reconnect = config.getBool(settings_config_prefix + ".background_reconnect", false), - }); - - pool = std::make_shared( - mysqlxx::PoolFactory::instance().get(config, settings_config_prefix)); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL dictionary source configuration must use a named collection"); } if (dictionary_configuration->query.empty() && dictionary_configuration->table.empty()) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 516bab8a483b..e48f858e22fd 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -29,6 +29,8 @@ namespace Setting extern const SettingsUInt64 postgresql_connection_pool_retries; extern const SettingsUInt64 postgresql_connection_pool_size; extern const SettingsUInt64 postgresql_connection_pool_wait_timeout; + extern const SettingsSSLMode postgresql_connection_pool_ssl_mode; + extern const SettingsString postgresql_connection_pool_ssl_root_cert; } namespace ErrorCodes @@ -327,6 +329,8 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) settings[Setting::postgresql_connection_pool_retries], settings[Setting::postgresql_connection_pool_auto_close_connection], settings[Setting::postgresql_connection_attempt_timeout], + static_cast(settings[Setting::postgresql_connection_pool_ssl_mode]), + static_cast(settings[Setting::postgresql_connection_pool_ssl_root_cert]), bg_reconnect); diff --git a/src/Storages/MySQL/MySQLHelpers.cpp b/src/Storages/MySQL/MySQLHelpers.cpp index cf7f86569fe9..dc63ddf6f8b6 100644 --- a/src/Storages/MySQL/MySQLHelpers.cpp +++ b/src/Storages/MySQL/MySQLHelpers.cpp @@ -26,8 +26,9 @@ mysqlxx::PoolWithFailover createMySQLPoolWithFailover(const StorageMySQL::Config return createMySQLPoolWithFailover( configuration.database, configuration.addresses, configuration.username, configuration.password, - configuration.ssl_ca, configuration.ssl_cert, - configuration.ssl_key, mysql_settings); + configuration.ssl_ca, configuration.ssl_cert, configuration.ssl_key, + configuration.ssl_mode, + mysql_settings); } mysqlxx::PoolWithFailover createMySQLPoolWithFailover( @@ -38,6 +39,7 @@ mysqlxx::PoolWithFailover createMySQLPoolWithFailover( const std::string & ssl_ca, const std::string & ssl_cert, const std::string & ssl_key, + MySQLSSLMode ssl_mode, const MySQLSettings & mysql_settings) { if (!mysql_settings[MySQLSetting::connection_pool_size]) @@ -45,6 +47,7 @@ mysqlxx::PoolWithFailover createMySQLPoolWithFailover( return mysqlxx::PoolWithFailover( database, addresses, username, password, ssl_ca, ssl_cert, ssl_key, + ssl_mode, MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, static_cast(mysql_settings[MySQLSetting::connection_pool_size]), mysql_settings[MySQLSetting::connection_max_tries], diff --git a/src/Storages/MySQL/MySQLHelpers.h b/src/Storages/MySQL/MySQLHelpers.h index f49bf97a1ee0..5330661ad950 100644 --- a/src/Storages/MySQL/MySQLHelpers.h +++ b/src/Storages/MySQL/MySQLHelpers.h @@ -21,6 +21,7 @@ mysqlxx::PoolWithFailover createMySQLPoolWithFailover( const std::string & ssl_ca, const std::string & ssl_cert, const std::string & ssl_key, + MySQLSSLMode ssl_mode, const MySQLSettings & mysql_settings); } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 061727b988bb..84f10921e687 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -579,7 +580,9 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) configuration.port, configuration.username, configuration.password, - args.getContext()->getSettingsRef()[Setting::postgresql_connection_attempt_timeout]); + args.getContext()->getSettingsRef()[Setting::postgresql_connection_attempt_timeout], + configuration.ssl_mode.value_or(SSLMode::PREFER), + configuration.ssl_root_cert); bool has_settings = args.storage_def->settings; auto postgresql_replication_settings = std::make_unique(); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index b9dec5f410b0..a742fcd9bd22 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -277,7 +278,7 @@ StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult( { StorageMySQL::Configuration configuration; - ValidateKeysMultiset optional_arguments = {"replace_query", "on_duplicate_clause", "addresses_expr", "host", "hostname", "port", "ssl_ca", "ssl_cert", "ssl_key"}; + ValidateKeysMultiset optional_arguments = {"replace_query", "on_duplicate_clause", "addresses_expr", "host", "hostname", "port", "ssl_ca", "ssl_cert", "ssl_key", "ssl_root_cert", "ssl_mode"}; auto mysql_settings_names = storage_settings.getAllRegisteredNames(); for (const auto & name : mysql_settings_names) optional_arguments.insert(name); @@ -304,6 +305,8 @@ StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult( configuration.username = named_collection.getAny({"username", "user"}); configuration.password = named_collection.get("password"); configuration.database = named_collection.getAny({"db", "database"}); + configuration.ssl_mode = SettingFieldMySQLSSLModeTraits::fromString(named_collection.getOrDefault("ssl_mode", "prefer")); + configuration.ssl_root_cert = named_collection.getOrDefault("ssl_root_cert", ""); if (require_table) configuration.table = named_collection.get("table"); configuration.replace_query = named_collection.getOrDefault("replace_query", false); diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 95ae89bb1a3f..1ee7501631b9 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -6,6 +6,7 @@ #include #include +#include namespace Poco { @@ -64,6 +65,8 @@ class StorageMySQL final : public IStorage, WithContext String ssl_ca; String ssl_cert; String ssl_key; + MySQLSSLMode ssl_mode = MySQLSSLMode::PREFER; + String ssl_root_cert; bool replace_query = false; String on_duplicate_clause; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5f489fa21472..eb55ccbafc82 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -64,6 +64,8 @@ namespace Setting extern const SettingsUInt64 postgresql_connection_pool_retries; extern const SettingsUInt64 postgresql_connection_pool_size; extern const SettingsUInt64 postgresql_connection_pool_wait_timeout; + extern const SettingsSSLMode postgresql_connection_pool_ssl_mode; + extern const SettingsString postgresql_connection_pool_ssl_root_cert; } namespace ErrorCodes @@ -537,7 +539,7 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult required_arguments.insert("table"); validateNamedCollection>( - named_collection, required_arguments, {"schema", "on_conflict", "addresses_expr", "host", "hostname", "port", "use_table_cache"}); + named_collection, required_arguments, {"schema", "on_conflict", "addresses_expr", "host", "hostname", "port", "use_table_cache", "ssl_root_cert", "ssl_mode"}); configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); if (configuration.addresses_expr.empty()) @@ -560,6 +562,11 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult configuration.table = named_collection.get("table"); configuration.schema = named_collection.getOrDefault("schema", ""); configuration.on_conflict = named_collection.getOrDefault("on_conflict", ""); + const String ssl_mode = named_collection.getOrDefault("ssl_mode", ""); + if (!ssl_mode.empty()) { + configuration.ssl_mode = SettingFieldSSLModeTraits::fromString(ssl_mode); + } + configuration.ssl_root_cert = named_collection.getOrDefault("ssl_root_cert", ""); return configuration; } @@ -624,7 +631,9 @@ void registerStoragePostgreSQL(StorageFactory & factory) settings[Setting::postgresql_connection_pool_wait_timeout], settings[Setting::postgresql_connection_pool_retries], settings[Setting::postgresql_connection_pool_auto_close_connection], - settings[Setting::postgresql_connection_attempt_timeout]); + settings[Setting::postgresql_connection_attempt_timeout], + static_cast(settings[Setting::postgresql_connection_pool_ssl_mode]), + static_cast(settings[Setting::postgresql_connection_pool_ssl_root_cert])); return std::make_shared( args.table_id, diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 487e890a5e34..b67aeb9e52a1 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -3,6 +3,7 @@ #include "config.h" #if USE_LIBPQXX +#include #include #include @@ -56,6 +57,8 @@ class StoragePostgreSQL final : public IStorage String username = "default"; String password; String database; + std::optional ssl_mode; + String ssl_root_cert; String table; String schema; String on_conflict; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index c362dc0ebd8b..caa51c08e1dd 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -23,6 +23,8 @@ namespace Setting extern const SettingsUInt64 postgresql_connection_pool_retries; extern const SettingsUInt64 postgresql_connection_pool_size; extern const SettingsUInt64 postgresql_connection_pool_wait_timeout; + extern const SettingsSSLMode postgresql_connection_pool_ssl_mode; + extern const SettingsString postgresql_connection_pool_ssl_root_cert; } namespace ErrorCodes @@ -92,7 +94,9 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, Contex settings[Setting::postgresql_connection_pool_wait_timeout], settings[Setting::postgresql_connection_pool_retries], settings[Setting::postgresql_connection_pool_auto_close_connection], - settings[Setting::postgresql_connection_attempt_timeout]); + settings[Setting::postgresql_connection_attempt_timeout], + static_cast(settings[Setting::postgresql_connection_pool_ssl_mode]), + static_cast(settings[Setting::postgresql_connection_pool_ssl_root_cert])); } } From 6178a04a28ee894cc173a6a9b5cd1c331d12574a Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 15 Dec 2025 14:36:23 +0100 Subject: [PATCH 22/49] Fix: stop wiping secrets from remote calls Remote calls (like `azureBlobStorageCluster`) format `AST` into `String` using `formatWithSecretsOneLine`. This function wipes sensitive data, which can lead to incorrect remote call. This commit makes wiping optional and stops it during the remote calls. --- src/Parsers/IAST.cpp | 12 ++++++++---- src/Parsers/IAST.h | 5 +++-- src/Storages/IStorageCluster.cpp | 2 +- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index a9923f6e00d6..cdd7e9e5062b 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -167,7 +167,8 @@ String IAST::formatWithPossiblyHidingSensitiveData( bool show_secrets, bool print_pretty_type_names, IdentifierQuotingRule identifier_quoting_rule, - IdentifierQuotingStyle identifier_quoting_style) const + IdentifierQuotingStyle identifier_quoting_style, + bool wipe_sensitive_data) const { WriteBufferFromOwnString buf; FormatSettings settings(one_line); @@ -176,7 +177,9 @@ String IAST::formatWithPossiblyHidingSensitiveData( settings.identifier_quoting_rule = identifier_quoting_rule; settings.identifier_quoting_style = identifier_quoting_style; format(buf, settings); - return wipeSensitiveDataAndCutToLength(buf.str(), max_length); + if (wipe_sensitive_data) + wipeSensitiveDataAndCutToLength(buf.str(), max_length); + return buf.str(); } String IAST::formatForLogging(size_t max_length) const @@ -201,7 +204,7 @@ String IAST::formatForErrorMessage() const /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks); } -String IAST::formatWithSecretsOneLine() const +String IAST::formatWithSecretsOneLine(bool wipe_sensitive_data) const { return formatWithPossiblyHidingSensitiveData( /*max_length=*/0, @@ -209,7 +212,8 @@ String IAST::formatWithSecretsOneLine() const /*show_secrets=*/true, /*print_pretty_type_names=*/false, /*identifier_quoting_rule=*/IdentifierQuotingRule::WhenNecessary, - /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks); + /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks, + /*wipe_sensitive_data=*/wipe_sensitive_data); } String IAST::formatWithSecretsMultiLine() const diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 598920439f96..a74c7a73c82d 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -286,7 +286,8 @@ class IAST : public std::enable_shared_from_this, public TypePromotion, public TypePromotion( std::vector{try_results.front()}, - query_to_send->formatWithSecretsOneLine(), + query_to_send->formatWithSecretsOneLine(false), getOutputHeader(), new_context, /*throttler=*/nullptr, From 3db5eef229124c528ee6346079452ef3526d5fd9 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 15 Dec 2025 15:58:23 +0100 Subject: [PATCH 23/49] Added support for protected users Add an extra flag to each user, called "Protected". Users with the "Protected" flag can only be created, altered, removed, or have privileges granted/revoked by users with the extra `PROTECTED_ACCESS_MANAGEMENT` privilege. This privilege is only meant to be granted to our internal admin user, and the "Protected" flag will be given to the main service user, allowing it to exist as an SQL user (which is necessary to have less privileges than hardcoded XML users), but not be removable or alterable by cluster users. Changes: - Added `protected_flag` field to User entity and `isProtected()` method to IAccessEntity interface - Added `PROTECTED_ACCESS_MANAGEMENT` access type for controlling operations on protected users - Added `PROTECTED` keyword to SQL parser for CREATE USER statements - Updated all access storage implementations (Disk, Memory, Multiple, Replicated, AccessControl) to support CheckFunc parameter for validation during insert/remove operations - Updated interpreters to enforce PROTECTED_ACCESS_MANAGEMENT privilege: - InterpreterCreateUserQuery: Check when creating/altering protected users - InterpreterDropAccessEntityQuery: Check when dropping protected users - InterpreterMoveAccessEntityQuery: Check when moving protected users - InterpreterGrantQuery: Check when granting/revoking to protected users - Updated User equality comparison to include protected_flag - Updated SHOW CREATE USER to display PROTECTED keyword when applicable API Adaptations: - Vector-based insert/insertOrReplace methods don't support CheckFunc in this ClickHouse version, so manual checks were added before calling these methods - ContextAccess::checkGranteeIsAllowed signature differs from original patch (doesn't accept ContextPtr), so the protected check is enforced at the interpreter level instead Co-authored-by: Kevin Michel --- src/Access/AccessBackup.cpp | 3 +- src/Access/AccessControl.cpp | 8 ++-- src/Access/AccessControl.h | 4 +- src/Access/Common/AccessType.h | 1 + src/Access/DiskAccessStorage.cpp | 26 +++++++----- src/Access/DiskAccessStorage.h | 8 ++-- src/Access/IAccessEntity.h | 2 + src/Access/IAccessStorage.cpp | 18 ++++++-- src/Access/IAccessStorage.h | 11 +++-- src/Access/MemoryAccessStorage.cpp | 26 +++++++----- src/Access/MemoryAccessStorage.h | 8 ++-- src/Access/MultipleAccessStorage.cpp | 8 ++-- src/Access/MultipleAccessStorage.h | 4 +- src/Access/ReplicatedAccessStorage.cpp | 9 +++- src/Access/ReplicatedAccessStorage.h | 4 +- src/Access/User.cpp | 3 +- src/Access/User.h | 2 + .../Access/InterpreterCreateUserQuery.cpp | 42 +++++++++++++++++++ .../InterpreterDropAccessEntityQuery.cpp | 29 ++++++++++++- .../Access/InterpreterGrantQuery.cpp | 4 ++ .../InterpreterMoveAccessEntityQuery.cpp | 16 +++++++ ...InterpreterShowCreateAccessEntityQuery.cpp | 3 ++ src/Parsers/Access/ASTCreateUserQuery.cpp | 3 ++ src/Parsers/Access/ASTCreateUserQuery.h | 1 + src/Parsers/Access/ParserCreateUserQuery.cpp | 8 ++++ src/Parsers/CommonParsers.h | 1 + 26 files changed, 196 insertions(+), 56 deletions(-) diff --git a/src/Access/AccessBackup.cpp b/src/Access/AccessBackup.cpp index 0ab868c1589a..1cfe7545aca2 100644 --- a/src/Access/AccessBackup.cpp +++ b/src/Access/AccessBackup.cpp @@ -561,7 +561,8 @@ void restoreAccessEntitiesFromBackup( LOG_TRACE(log, "{}: Adding with UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), id); UUID existing_id; - if (destination_access_storage.insert(id, entity, replace_if_exists, throw_if_exists, &existing_id)) + auto check_func = [](const AccessEntityPtr &){}; // No check needed during restore + if (destination_access_storage.insert(id, entity, check_func, replace_if_exists, throw_if_exists, &existing_id)) { LOG_TRACE(log, "{}: Added successfully", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name)); restored_ids.emplace(id); diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index e17d9b2d9320..ebe4fae6d738 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -551,9 +551,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) +bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { - if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) + if (MultipleAccessStorage::insertImpl(id, entity, check_func, replace_if_exists, throw_if_exists, conflicting_id)) { changes_notifier->sendNotifications(); return true; @@ -561,9 +561,9 @@ bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, return false; } -bool AccessControl::removeImpl(const UUID & id, bool throw_if_not_exists) +bool AccessControl::removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) { - bool removed = MultipleAccessStorage::removeImpl(id, throw_if_not_exists); + bool removed = MultipleAccessStorage::removeImpl(id, check_func, throw_if_not_exists); if (removed) changes_notifier->sendNotifications(); return removed; diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 7395680288c3..f55401c0d6e5 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -268,8 +268,8 @@ class AccessControl : public MultipleAccessStorage class CustomSettingsPrefixes; class PasswordComplexityRules; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; - bool removeImpl(const UUID & id, bool throw_if_not_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; + bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::unique_ptr context_access_cache; diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 13a9911c702e..1fb1011122a1 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -284,6 +284,7 @@ enum class AccessType : uint8_t M(ALTER_SETTINGS_PROFILE, "ALTER PROFILE", GLOBAL, ACCESS_MANAGEMENT) \ M(DROP_SETTINGS_PROFILE, "DROP PROFILE", GLOBAL, ACCESS_MANAGEMENT) \ M(ALLOW_SQL_SECURITY_NONE, "CREATE SQL SECURITY NONE, ALLOW SQL SECURITY NONE, SQL SECURITY NONE, SECURITY NONE", GLOBAL, ACCESS_MANAGEMENT) \ + M(PROTECTED_ACCESS_MANAGEMENT, "", GLOBAL, ACCESS_MANAGEMENT) \ M(SHOW_USERS, "SHOW CREATE USER", GLOBAL, SHOW_ACCESS) \ M(SHOW_ROLES, "SHOW CREATE ROLE", GLOBAL, SHOW_ACCESS) \ M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", TABLE, SHOW_ACCESS) \ diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 1d263fc5c6fe..7e1ec085f75a 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -415,7 +415,7 @@ void DiskAccessStorage::setAllInMemory(const std::vector & ids_to_keep) @@ -425,7 +425,7 @@ void DiskAccessStorage::removeAllExceptInMemory(const boost::container::flat_set const auto & id = it->first; ++it; /// We must go to the next element in the map `entries_by_id` here because otherwise removeNoLock() can invalidate our iterator. if (!ids_to_keep.contains(id)) - (void)removeNoLock(id, /* throw_if_not_exists */ true, /* write_on_disk= */ false); + (void)removeNoLock(id, CheckFunc{}, /* throw_if_not_exists */ true, /* write_on_disk= */ false); } } @@ -504,15 +504,18 @@ std::optional> DiskAccessStorage::readNameWi } -bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) +bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id, /* write_on_disk = */ true); + return insertNoLock(id, new_entity, check_func, replace_if_exists, throw_if_exists, conflicting_id, /* write_on_disk = */ true); } -bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) +bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) { + if (check_func) + check_func(new_entity); + const String & name = new_entity->getName(); AccessEntityType type = new_entity->getType(); @@ -565,7 +568,7 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne if (name_collision && (id_by_name != id)) { assert(replace_if_exists); - removeNoLock(id_by_name, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT + removeNoLock(id_by_name, CheckFunc{}, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT } if (id_collision) @@ -590,7 +593,7 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne return true; } - removeNoLock(id, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT + removeNoLock(id, CheckFunc{}, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT } /// Do insertion. @@ -609,14 +612,14 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne } -bool DiskAccessStorage::removeImpl(const UUID & id, bool throw_if_not_exists) +bool DiskAccessStorage::removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) { std::lock_guard lock{mutex}; - return removeNoLock(id, throw_if_not_exists, /* write_on_disk= */ true); + return removeNoLock(id, check_func, throw_if_not_exists, /* write_on_disk= */ true); } -bool DiskAccessStorage::removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) +bool DiskAccessStorage::removeNoLock(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists, bool write_on_disk) { auto it = entries_by_id.find(id); if (it == entries_by_id.end()) @@ -627,6 +630,9 @@ bool DiskAccessStorage::removeNoLock(const UUID & id, bool throw_if_not_exists, return false; } + if (check_func && it->second.entity) + check_func(it->second.entity); + Entry & entry = it->second; AccessEntityType type = entry.type; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 40f2017dd979..43c55b1ca9fd 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -40,8 +40,8 @@ class DiskAccessStorage : public IAccessStorage std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; - bool removeImpl(const UUID & id, bool throw_if_not_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; + bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; bool readLists() TSA_REQUIRES(mutex); @@ -54,9 +54,9 @@ class DiskAccessStorage : public IAccessStorage void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS; void stopListsWritingThread(); - bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) TSA_REQUIRES(mutex); + bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) TSA_REQUIRES(mutex); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); - bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); + bool removeNoLock(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); AccessEntityPtr readAccessEntityFromDisk(const UUID & id) const; void writeAccessEntityToDisk(const UUID & id, const IAccessEntity & entity) const; diff --git a/src/Access/IAccessEntity.h b/src/Access/IAccessEntity.h index 09c5eb6cf10e..30ef4238ea50 100644 --- a/src/Access/IAccessEntity.h +++ b/src/Access/IAccessEntity.h @@ -31,6 +31,8 @@ struct IAccessEntity virtual void setName(const String & name_) { name = name_; } const String & getName() const { return name; } + virtual bool isProtected() const { return false; } + friend bool operator ==(const IAccessEntity & lhs, const IAccessEntity & rhs) { return lhs.equal(rhs); } friend bool operator !=(const IAccessEntity & lhs, const IAccessEntity & rhs) { return !(lhs == rhs); } diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 3ac24ff1e820..d80db6e174e0 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -210,7 +210,12 @@ std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { - return insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id); + return insert(id, entity, CheckFunc{}, replace_if_exists, throw_if_exists, conflicting_id); +} + +bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) +{ + return insertImpl(id, entity, check_func, replace_if_exists, throw_if_exists, conflicting_id); } @@ -304,7 +309,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vectorgetType(), entity->getName()); @@ -314,7 +319,12 @@ bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bo bool IAccessStorage::remove(const UUID & id, bool throw_if_not_exists) { - return removeImpl(id, throw_if_not_exists); + return remove(id, CheckFunc{}, throw_if_not_exists); +} + +bool IAccessStorage::remove(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) +{ + return removeImpl(id, check_func, throw_if_not_exists); } @@ -395,7 +405,7 @@ std::vector IAccessStorage::tryRemove(const std::vector & ids) } -bool IAccessStorage::removeImpl(const UUID & id, bool throw_if_not_exists) +bool IAccessStorage::removeImpl(const UUID & id, const CheckFunc &, bool throw_if_not_exists) { if (isReadOnly(id)) { diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index bb38e8ec3d1c..f29dd465ca70 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -158,10 +158,14 @@ class IAccessStorage : public boost::noncopyable std::vector> readAllWithIDs(AccessEntityType type) const; + using UpdateFunc = std::function; + using CheckFunc = std::function; + /// Inserts an entity to the storage. Returns ID of a new entry in the storage. /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr); + bool insert(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr); bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); @@ -176,6 +180,7 @@ class IAccessStorage : public boost::noncopyable std::vector insertOrReplace(const std::vector & multiple_entities); /// Removes an entity from the storage. Throws an exception if couldn't remove. + bool remove(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists = true); bool remove(const UUID & id, bool throw_if_not_exists = true); std::vector remove(const std::vector & ids, bool throw_if_not_exists = true); @@ -185,8 +190,6 @@ class IAccessStorage : public boost::noncopyable /// Removes multiple entities from the storage. Returns the list of successfully dropped. std::vector tryRemove(const std::vector & ids); - using UpdateFunc = std::function; - /// Updates an entity stored in the storage. Throws an exception if couldn't update. bool update(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists = true); std::vector update(const std::vector & ids, const UpdateFunc & update_func, bool throw_if_not_exists = true); @@ -237,8 +240,8 @@ class IAccessStorage : public boost::noncopyable virtual std::vector findAllImpl() const; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); - virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); + virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); + virtual bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl( const Credentials & credentials, diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index e36143de149d..6cb5df377ac9 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -61,15 +61,18 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) +bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id); + return insertNoLock(id, new_entity, check_func, replace_if_exists, throw_if_exists, conflicting_id); } -bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) +bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { + if (check_func) + check_func(new_entity); + const String & name = new_entity->getName(); AccessEntityType type = new_entity->getType(); @@ -116,7 +119,7 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & if (name_collision && (id_by_name != id)) { assert(replace_if_exists); - removeNoLock(id_by_name, /* throw_if_not_exists= */ true); // NOLINT + removeNoLock(id_by_name, CheckFunc{}, /* throw_if_not_exists= */ true); // NOLINT } if (id_collision) @@ -138,7 +141,7 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & } return true; } - removeNoLock(id, /* throw_if_not_exists= */ true); // NOLINT + removeNoLock(id, CheckFunc{}, /* throw_if_not_exists= */ true); // NOLINT } /// Do insertion. @@ -151,14 +154,14 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & } -bool MemoryAccessStorage::removeImpl(const UUID & id, bool throw_if_not_exists) +bool MemoryAccessStorage::removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) { std::lock_guard lock{mutex}; - return removeNoLock(id, throw_if_not_exists); + return removeNoLock(id, check_func, throw_if_not_exists); } -bool MemoryAccessStorage::removeNoLock(const UUID & id, bool throw_if_not_exists) +bool MemoryAccessStorage::removeNoLock(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) { auto it = entries_by_id.find(id); if (it == entries_by_id.end()) @@ -169,6 +172,9 @@ bool MemoryAccessStorage::removeNoLock(const UUID & id, bool throw_if_not_exists return false; } + if (check_func && it->second.entity) + check_func(it->second.entity); + Entry & entry = it->second; const String & name = entry.entity->getName(); AccessEntityType type = entry.entity->getType(); @@ -248,7 +254,7 @@ void MemoryAccessStorage::removeAllExceptNoLock(const boost::container::flat_set const auto & id = it->first; ++it; /// We must go to the next element in the map `entries_by_id` here because otherwise removeNoLock() can invalidate our iterator. if (!ids_to_keep.contains(id)) - removeNoLock(id, /* throw_if_not_exists */ true); // NOLINT + removeNoLock(id, CheckFunc{}, /* throw_if_not_exists */ true); // NOLINT } } @@ -280,7 +286,7 @@ void MemoryAccessStorage::setAll(const std::vector findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; - bool removeImpl(const UUID & id, bool throw_if_not_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; + bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); - bool removeNoLock(const UUID & id, bool throw_if_not_exists); + bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); + bool removeNoLock(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); void removeAllExceptNoLock(const std::vector & ids_to_keep); diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index c7247900b1c2..6baae18c48f0 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -374,7 +374,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) +bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::shared_ptr storage_for_insertion; @@ -397,7 +397,7 @@ bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & getStorageName()); } - if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) + if (storage_for_insertion->insert(id, entity, check_func, replace_if_exists, throw_if_exists, conflicting_id)) { std::lock_guard lock{mutex}; ids_cache.set(id, storage_for_insertion); @@ -408,10 +408,10 @@ bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & } -bool MultipleAccessStorage::removeImpl(const UUID & id, bool throw_if_not_exists) +bool MultipleAccessStorage::removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) { if (auto storage = findStorage(id)) - return storage->remove(id, throw_if_not_exists); + return storage->remove(id, check_func, throw_if_not_exists); if (throw_if_not_exists) throwNotFound(id, getStorageName()); diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 82492542e853..1d5fc3fc13d1 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -70,8 +70,8 @@ class MultipleAccessStorage : public IAccessStorage std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; - bool removeImpl(const UUID & id, bool throw_if_not_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; + bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, const ClientInfo & client_info, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index a40fbfcae627..ddb74cf5d713 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -75,13 +75,18 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) +bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { + if (check_func) + check_func(new_entity); return replicator.insertEntity(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id); } -bool ReplicatedAccessStorage::removeImpl(const UUID & id, bool throw_if_not_exists) +bool ReplicatedAccessStorage::removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) { + auto entity = tryRead(id); + if (check_func && entity) + check_func(entity); return replicator.removeEntity(id, throw_if_not_exists); } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index d07cb2acb50b..6f7505311e36 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -33,8 +33,8 @@ class ReplicatedAccessStorage : public IAccessStorage bool isBackupAllowed() const override { return backup_allowed; } private: - bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; - bool removeImpl(const UUID & id, bool throw_if_not_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; + bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional findImpl(AccessEntityType type, const String & name) const override; diff --git a/src/Access/User.cpp b/src/Access/User.cpp index 1c92f467003a..fbcda0eee83e 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -19,7 +19,8 @@ bool User::equal(const IAccessEntity & other) const return (authentication_methods == other_user.authentication_methods) && (allowed_client_hosts == other_user.allowed_client_hosts) && (access == other_user.access) && (granted_roles == other_user.granted_roles) && (default_roles == other_user.default_roles) - && (settings == other_user.settings) && (grantees == other_user.grantees) && (default_database == other_user.default_database); + && (settings == other_user.settings) && (grantees == other_user.grantees) && (default_database == other_user.default_database) + && (protected_flag == other_user.protected_flag); } void User::setName(const String & name_) diff --git a/src/Access/User.h b/src/Access/User.h index f54e74a305d6..3ecb245f2f5a 100644 --- a/src/Access/User.h +++ b/src/Access/User.h @@ -23,6 +23,7 @@ struct User : public IAccessEntity SettingsProfileElements settings; RolesOrUsersSet grantees = RolesOrUsersSet::AllTag{}; String default_database; + bool protected_flag = false; bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } @@ -37,6 +38,7 @@ struct User : public IAccessEntity void removeDependencies(const std::unordered_set & ids) override; void clearAllExceptDependencies() override; + bool isProtected() const override { return protected_flag; } bool isBackupAllowed() const override { return settings.isBackupAllowed(); } }; diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 7e8f08f18c10..9fbea43431cb 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -12,6 +12,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -199,6 +202,10 @@ BlockIO InterpreterCreateUserQuery::execute() if (query.new_name && !query.alter) access->checkAccess(AccessType::CREATE_USER, *query.new_name); + // Statements containing the PROTECTED keyword require an extra privilege + if (query.protected_flag) + access->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); + bool implicit_no_password_allowed = access_control.isImplicitNoPasswordAllowed(); bool no_password_allowed = access_control.isNoPasswordAllowed(); bool plaintext_password_allowed = access_control.isPlaintextPasswordAllowed(); @@ -258,11 +265,30 @@ BlockIO InterpreterCreateUserQuery::execute() auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr { auto updated_user = typeid_cast>(entity->clone()); + + // Check if modifying a protected user + if (entity->isProtected() && !query.protected_flag) + { + // Removing protected flag requires PROTECTED_ACCESS_MANAGEMENT + access->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); + } + else if (!entity->isProtected() && query.protected_flag) + { + // Adding protected flag requires PROTECTED_ACCESS_MANAGEMENT + access->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); + } + else if (entity->isProtected()) + { + // Modifying protected user requires PROTECTED_ACCESS_MANAGEMENT + access->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); + } + updateUserFromQueryImpl( *updated_user, query, authentication_methods, {}, default_roles_from_query, settings_from_query, grantees_from_query, global_valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed, getContext()->getServerSettings()[ServerSetting::max_authentication_methods_per_user]); + updated_user->protected_flag = query.protected_flag; return updated_user; }; @@ -286,9 +312,25 @@ BlockIO InterpreterCreateUserQuery::execute() global_valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed, getContext()->getServerSettings()[ServerSetting::max_authentication_methods_per_user]); + new_user->protected_flag = query.protected_flag; new_users.emplace_back(std::move(new_user)); } + // Check if replacing protected users (for insertOrReplace) + if (query.or_replace) + { + for (const auto & name : names) + { + if (auto existing_user = storage->tryRead(name)) + { + if (existing_user->isProtected()) + { + access->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); + } + } + } + } + if (!query.storage_name.empty()) { for (const auto & name : names) diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index 6622b9b8ff10..d2ab1fe4745c 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -2,7 +2,10 @@ #include #include +#include #include +#include +#include #include #include #include @@ -42,10 +45,32 @@ BlockIO InterpreterDropAccessEntityQuery::execute() storage = storage_ptr.get(); } + // Create CheckFunc to validate protected users + auto access_ptr = getContext()->getAccess(); + auto check_func = [access_ptr](const AccessEntityPtr & entity) + { + if (entity->isProtected()) + { + access_ptr->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); + } + }; + if (query.if_exists) - storage->tryRemove(storage->find(query.type, names)); + { + auto ids = storage->find(query.type, names); + for (const auto & id : ids) + { + storage->remove(id, check_func); + } + } else - storage->remove(storage->getIDs(query.type, names)); + { + auto ids = storage->getIDs(query.type, names); + for (const auto & id : ids) + { + storage->remove(id, check_func); + } + } }; if (query.type == AccessEntityType::USER) diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 0ea3fa0aa82c..c73a0abb9b55 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -538,6 +540,8 @@ BlockIO InterpreterGrantQuery::execute() /// Update roles and users listed in `grantees`. auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr { + if (entity->isProtected()) + current_user_access->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); auto clone = entity->clone(); if (query.current_grants) grantCurrentGrants(*clone, new_rights, elements_to_revoke); diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp index ba3d749fb591..f5545bb528d6 100644 --- a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -4,6 +4,9 @@ #include #include #include +#include +#include +#include #include #include @@ -39,6 +42,19 @@ BlockIO InterpreterMoveAccessEntityQuery::execute() if (!source_storage->exists(ids)) throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "All access entities must be from the same storage in order to be moved"); + // Check if moving protected users + if (query.type == AccessEntityType::USER) + { + for (const auto & id : ids) + { + auto user = access_control.tryRead(id); + if (user && user->isProtected()) + { + getContext()->checkAccess(AccessFlags{AccessType::PROTECTED_ACCESS_MANAGEMENT}); + } + } + } + access_control.moveAccessEntities(ids, source_storage->getStorageName(), query.storage_name); return {}; } diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 03d56747c1bd..e9894d761352 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -94,6 +94,9 @@ namespace query->default_database = ast; } + if (user.isProtected()) + query->protected_flag = true; + return query; } diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index ebc9df2a218f..bc9433b3f559 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -254,6 +254,9 @@ void ASTCreateUserQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & f if (global_valid_until) formatValidUntil(*global_valid_until, ostr, format); + if (protected_flag && !alter) + ostr << " PROTECTED"; + if (hosts) formatHosts(nullptr, *hosts, ostr, format); if (add_hosts) diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index ff1ec4f12705..32fa3cf96c7d 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -50,6 +50,7 @@ class ASTCreateUserQuery : public IAST, public ASTQueryWithOnCluster bool reset_authentication_methods_to_new = false; bool add_identified_with = false; bool replace_authentication_methods = false; + bool protected_flag = false; std::shared_ptr names; std::optional new_name; diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 1f3b0089410b..564a41d469b5 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -593,6 +593,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec String cluster; String storage_name; bool reset_authentication_methods_to_new = false; + bool protected_flag = false; bool parsed_identified_with = false; bool parsed_add_identified_with = false; @@ -703,6 +704,12 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } } + if (!alter && ParserKeyword{Keyword::PROTECTED}.ignore(pos, expected)) + { + protected_flag = true; + continue; + } + break; } @@ -745,6 +752,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->reset_authentication_methods_to_new = reset_authentication_methods_to_new; query->add_identified_with = parsed_add_identified_with; query->replace_authentication_methods = parsed_identified_with; + query->protected_flag = protected_flag; for (const auto & authentication_method : query->authentication_methods) { diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 4ff75fe57407..063b37fc737c 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -394,6 +394,7 @@ namespace DB MR_MACROS(PROFILE, "PROFILE") \ MR_MACROS(PROFILES, "PROFILES") \ MR_MACROS(PROJECTION, "PROJECTION") \ + MR_MACROS(PROTECTED, "PROTECTED") \ MR_MACROS(PROTOBUF, "Protobuf") \ MR_MACROS(FILES, "Files") \ MR_MACROS(PULL, "PULL") \ From 0a265dab26e98e3fbf83da34f1c5b0b8fc13d9f6 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 15 Dec 2025 16:17:18 +0100 Subject: [PATCH 24/49] Fix IPv6 S3 object storage host The AWS SDK does not fully support IPv6 in hostnames, mainly because the escaping brackets are not parsed and removed at the right time. Co-authored-by: Kevin Michel --- contrib/aws | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index 39c331979c58..30c8374334bb 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 39c331979c58146578dfedae56449cebf177125a +Subproject commit 30c8374334bbfa1c76f336e9827f178337f27703 From 5e66d02da4a7f5c9a308505312f2aed48d65d6d6 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 16 Dec 2025 11:36:24 +0100 Subject: [PATCH 25/49] Fix IPv6 Azure object storage host The Azure SDK does not fully support IPv6 in hostnames, mainly because the escaping brackets are not parsed and removed at the right time. The second fix is to detect IPv6 in hosts and validate them by parsing them instead of rejecting them because they don't look like DNS name segments. This change updates the URL validation regex pattern in validateStorageAccountUrl() to accept IPv6 addresses in brackets format (e.g., [2001:db8::1]) and also adds support for optional port numbers in the URL. The azure submodule has been updated to include the corresponding fixes in the Azure SDK for C++. Co-authored-by: Kevin Michel --- contrib/azure | 2 +- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/azure b/contrib/azure index 3278daf75acf..ceb5b8534eb6 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 3278daf75acfb195b7746ef50c82c0a35abb5e02 +Subproject commit ceb5b8534eb67e8490ccf28cab228b6882882e23 diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 8a4877244dab..c4bb0a31ce0a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -96,7 +96,7 @@ namespace AzureBlobStorage static void validateStorageAccountUrl(const String & storage_account_url) { - const auto * storage_account_url_pattern_str = R"(http(()|s)://[a-z0-9-.:]+(()|/)[a-z0-9]*(()|/))"; + const auto * storage_account_url_pattern_str = R"(http(()|s)://(\[[a-fA-F0-9:]+\]|[a-z0-9-.]+)(:\d+)?(()|/)[a-z0-9]*(()|/))"; static const RE2 storage_account_url_pattern(storage_account_url_pattern_str); if (!re2::RE2::FullMatch(storage_account_url, storage_account_url_pattern)) From f569fe1ffe15307ca55fea904cdc0f6dcd74248b Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 16 Dec 2025 11:50:20 +0100 Subject: [PATCH 26/49] Add support for Azure object storage path prefix Azure storage does not support having a prefix before the stored object key. This makes it unusable for us when we use shared buckets with prefixes for projects and backup sites. This change adds support for a `storage_prefix` configuration option that can be used with `storage_account_url` or `connection_string` configuration methods. The prefix is prepended to all blob operations, allowing multiple ClickHouse instances or projects to share the same Azure container by using different path prefixes. Previously, prefix could only be specified via the `endpoint` configuration method by including it in the endpoint URL path. This change makes prefix configuration more flexible and readable, especially for shared container scenarios. Co-authored-by: Kevin Michel --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index c4bb0a31ce0a..b4f705171dab 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -515,6 +515,9 @@ Endpoint processEndpoint(const Poco::Util::AbstractConfiguration & config, const else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected either `storage_account_url` or `connection_string` or `endpoint` in config"); + if (config.has(config_prefix + ".storage_prefix")) + prefix = config.getString(config_prefix + ".storage_prefix"); + if (!container_name.empty()) validateContainerName(container_name); From c6db6f235c5cb3b15f2271971d996190f01cfa80 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 16 Dec 2025 15:31:56 +0100 Subject: [PATCH 27/49] Add Backup disk type This adds a new disk type that wraps an object storage disk and enables coordination with our backup system. When this disk type is used as a wrapper on top of an object storage disk, the files in object storage are not really deleted. Instead, a local file is created for each file in object storage that should be deleted. Our backup system will then be in charge to do garbage collection and only delete files that are not referenced by any backup. The file names of the deletion markers are like this: CLICKHOUSE_DIR/disks/remote_backup/backup/ESCAPED_OBJECT_PATH The object path is escaped to simplify directory management and uses the same escape mechanism that ClickHouse uses in many places to reference files (essentially escape encoding). In case Backup disk is wrapping a Cache disk then it lets the remove calls pass. Changes: - Added BackupObjectStorage wrapper class that intercepts delete operations and creates local deletion marker files - Added registerDiskBackup factory function to register the backup disk type - Extended IObjectStorage interface with getLayerName() and getWrappedObjectStorage() methods for generic layer traversal - Renamed getCacheLayersNames() to getLayersNames() to support both cache and backup layers - Added supportsLayers() method to IDisk interface - Updated DiskObjectStorage to support wrapping with backup layer - Updated MergeTreeData to track backup layers in addition to cache layers Adaptations from original patch: - Removed cloneObjectStorage() method (not in current IObjectStorage interface) - Fixed include order to prevent Poco namespace resolution issues - Updated all wrapper classes (ReadOnlyDiskWrapper, DiskEncrypted) to support the new layer interface This commit was applied from the patch file 0059-Backup_disk.patch Co-authored-by: Kevin Michel --- src/CMakeLists.txt | 1 + src/Disks/DiskEncrypted.h | 3 +- src/Disks/IDisk.h | 6 +- .../Backup/BackupObjectStorage.cpp | 77 ++++++++++ .../Backup/BackupObjectStorage.h | 140 ++++++++++++++++++ .../Backup/registerDiskBackup.cpp | 57 +++++++ .../Cached/CachedObjectStorage.h | 4 +- .../ObjectStorages/DiskObjectStorage.cpp | 6 + src/Disks/ObjectStorages/DiskObjectStorage.h | 9 +- .../ObjectStorages/DiskObjectStorageCache.cpp | 13 +- src/Disks/ObjectStorages/IObjectStorage.h | 4 + src/Disks/ReadOnlyDiskWrapper.h | 3 +- src/Disks/registerDisks.cpp | 4 + src/Storages/MergeTree/MergeTreeData.cpp | 16 +- 14 files changed, 326 insertions(+), 17 deletions(-) create mode 100644 src/Disks/ObjectStorages/Backup/BackupObjectStorage.cpp create mode 100644 src/Disks/ObjectStorages/Backup/BackupObjectStorage.h create mode 100644 src/Disks/ObjectStorages/Backup/registerDiskBackup.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 2dcf07466941..28a14bbe8eab 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -170,6 +170,7 @@ endif() add_headers_and_sources(dbms Databases/DataLake) +add_headers_and_sources(dbms Disks/ObjectStorages/Backup) add_headers_and_sources(dbms Disks/ObjectStorages/Cached) add_headers_and_sources(dbms Disks/ObjectStorages/Local) add_headers_and_sources(dbms Disks/ObjectStorages/Web) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index dac01956733c..a5de4a838129 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -319,7 +319,8 @@ class DiskEncrypted : public IDisk } bool supportsCache() const override { return delegate->supportsCache(); } - NameSet getCacheLayersNames() const override { return delegate->getCacheLayersNames(); } + bool supportsLayers() const override { return delegate->supportsLayers(); } + NameSet getLayersNames() const override { return delegate->getLayersNames(); } const String & getCacheName() const override { return delegate->getCacheName(); } bool isRemote() const override { return delegate->isRemote(); } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 0ae5979660ee..56d996450725 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -320,10 +320,12 @@ class IDisk : public Space virtual bool supportsCache() const { return false; } - virtual NameSet getCacheLayersNames() const + virtual bool supportsLayers() const { return false; } + + virtual NameSet getLayersNames() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Method `getCacheLayersNames()` is not implemented for disk: {}", + "Method `getLayersNames()` is not implemented for disk: {}", getDataSourceDescription().toString()); } diff --git a/src/Disks/ObjectStorages/Backup/BackupObjectStorage.cpp b/src/Disks/ObjectStorages/Backup/BackupObjectStorage.cpp new file mode 100644 index 000000000000..fc6ffccf920c --- /dev/null +++ b/src/Disks/ObjectStorages/Backup/BackupObjectStorage.cpp @@ -0,0 +1,77 @@ +#include "BackupObjectStorage.h" + +#include +#include +#include +#include + +namespace fs = std::filesystem; + +namespace DB +{ + +BackupObjectStorage::BackupObjectStorage( + const ObjectStoragePtr & object_storage_, const std::string & backup_base_path_, const std::string & backup_config_name_) + : object_storage(object_storage_) + , backup_base_path(backup_base_path_) + , backup_config_name(backup_config_name_) + , log(&Poco::Logger::get(getName())) +{ +} + +void BackupObjectStorage::removeObjectIfExists(const StoredObject & object) +{ + LOG_DEBUG(log, "removeObjectIfExists: {} -> {}", object.remote_path, object.local_path); + removeObjectImpl(object.remote_path); +} + +void BackupObjectStorage::removeObjectsIfExist(const StoredObjects & objects) +{ + for (const auto & object : objects) + { + LOG_DEBUG(log, "removeObjectsIfExist: {} -> {}", object.remote_path, object.local_path); + removeObjectImpl(object.remote_path); + } +} + +bool BackupObjectStorage::exists(const StoredObject & object) const +{ + return !isSoftDeleted(object.remote_path) && object_storage->exists(object); +} + +void BackupObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const +{ + RelativePathsWithMetadata all_children; + object_storage->listObjects(path, all_children, max_keys); + for (const auto & child : all_children) + { + if (!isSoftDeleted(child->getPath())) + { + children.push_back(child); + } + } +} + +bool BackupObjectStorage::isSoftDeleted(const std::string & object_path) const +{ + return FS::exists(getRemovedMarkerPath(object_path)); +} + +std::string BackupObjectStorage::getRemovedMarkerPath(const std::string & object_path) const +{ + return fs::path(backup_base_path) / escapeForFileName(object_path); +} + +void BackupObjectStorage::removeObjectImpl(const std::string & object_path) const +{ + const std::string removed_marker_path = getRemovedMarkerPath(object_path); + LOG_DEBUG(log, "adding removed marker: {}", removed_marker_path); + FS::createFile(removed_marker_path); +} +ObjectStorageKey +BackupObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const +{ + return object_storage->generateObjectKeyForPath(path, key_prefix); +} + +} diff --git a/src/Disks/ObjectStorages/Backup/BackupObjectStorage.h b/src/Disks/ObjectStorages/Backup/BackupObjectStorage.h new file mode 100644 index 000000000000..1bba176282f6 --- /dev/null +++ b/src/Disks/ObjectStorages/Backup/BackupObjectStorage.h @@ -0,0 +1,140 @@ +#pragma once + +#include +#include +#include + +namespace Poco +{ +class Logger; +} + +namespace DB +{ + +class BackupObjectStorage final : public IObjectStorage +{ +public: + BackupObjectStorage( + const ObjectStoragePtr & object_storage_, const std::string & backup_base_path_, const std::string & backup_config_name_); + + ObjectStorageType getType() const override { return object_storage->getType(); } + + std::string getCommonKeyPrefix() const override { return object_storage->getCommonKeyPrefix(); } + + void removeObjectIfExists(const StoredObject & object) override; + + virtual void removeObjectsIfExist(const StoredObjects & objects) override; + + bool exists(const StoredObject & object) const override; + + /// List objects recursively by certain prefix. + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; + + std::string getDescription() const override { return object_storage->getDescription(); } + + std::string getName() const override { return fmt::format("BackupObjectStorage-{}({})", backup_config_name, object_storage->getName()); } + + std::unique_ptr readObject( /// NOLINT + const StoredObject & object, + const ReadSettings & read_settings = ReadSettings{}, + std::optional read_hint = {}, + std::optional file_size = {}) const override + { + return object_storage->readObject(object, read_settings, read_hint, file_size); + } + + /// Open the file for write and return WriteBufferFromFileBase object. + virtual std::unique_ptr writeObject( /// NOLINT + const StoredObject & object, + WriteMode mode, + std::optional attributes = {}, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + const WriteSettings & write_settings = {}) override + { + return object_storage->writeObject(object, mode, attributes, buf_size, write_settings); + } + + virtual void copyObject( /// NOLINT + const StoredObject & object_from, + const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + std::optional object_to_attributes = {}) override + { + object_storage->copyObject(object_from, object_to, read_settings, write_settings, object_to_attributes); + } + + virtual void copyObjectToAnotherObjectStorage( /// NOLINT + const StoredObject & object_from, + const StoredObject & object_to, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + IObjectStorage & object_storage_to, + std::optional object_to_attributes = {}) override + { + object_storage->copyObjectToAnotherObjectStorage( + object_from, object_to, read_settings, write_settings, object_storage_to, object_to_attributes); + } + + bool areObjectKeysRandom() const override { return object_storage->areObjectKeysRandom(); } + + ObjectMetadata getObjectMetadata(const std::string & path) const override { return object_storage->getObjectMetadata(path); } + + void shutdown() override { object_storage->shutdown(); } + + void startup() override { object_storage->startup(); } + + void applyNewSettings( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + ContextPtr context, + const ApplyNewSettingsOptions & options) override + { + object_storage->applyNewSettings(config, config_prefix, context, options); + } + + String getObjectsNamespace() const override { return object_storage->getObjectsNamespace(); } + + bool isRemote() const override { return object_storage->isRemote(); } + + // This function is part of IObjectStorage but only used in CachedObjectStorage + // without any delegation, it probably shouldn't be part of the interface. + void removeCacheIfExists(const std::string & /*path_key_for_cache*/) override {} + + bool supportsCache() const override { return false; } + + std::string getUniqueId(const std::string & path) const override { return object_storage->getUniqueId(path); } + + bool isReadOnly() const override { return object_storage->isReadOnly(); } + + bool isWriteOnce() const override { return object_storage->isWriteOnce(); } + + std::optional getLayerName() const override { return {backup_config_name}; } + + ObjectStoragePtr getWrappedObjectStorage() override { return object_storage; } + + bool supportParallelWrite() const override { return object_storage->supportParallelWrite(); } + + ReadSettings patchSettings(const ReadSettings & read_settings) const override { return read_settings; } + + WriteSettings patchSettings(const WriteSettings & write_settings) const override { return write_settings; } + +private: + bool isSoftDeleted(const std::string & object_path) const; + + std::string getRemovedMarkerPath(const std::string & object_path) const; + + void removeObjectImpl(const std::string & object_path) const; + +public: + ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; + +private: + ObjectStoragePtr object_storage; + std::string backup_base_path; + std::string backup_config_name; + Poco::Logger * log; +}; + +} diff --git a/src/Disks/ObjectStorages/Backup/registerDiskBackup.cpp b/src/Disks/ObjectStorages/Backup/registerDiskBackup.cpp new file mode 100644 index 000000000000..e109e8d45011 --- /dev/null +++ b/src/Disks/ObjectStorages/Backup/registerDiskBackup.cpp @@ -0,0 +1,57 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +void registerDiskBackup(DiskFactory & factory, bool /* global_skip_access_check */) +{ + auto creator = [](const String & name, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context, + const DisksMap & map, + bool ,bool) -> DiskPtr + { + auto disk_name = config.getString(config_prefix + ".disk", ""); + if (disk_name.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk Backup requires `disk` field in config"); + + auto disk_it = map.find(disk_name); + if (disk_it == map.end()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot wrap disk `{}` with backup layer `{}`: there is no such disk (it should be initialized before backup disk)", + disk_name, name); + } + + auto backup_base_path = config.getString(config_prefix + ".path", fs::path(context->getPath()) / "disks" / name / "backup/"); + if (!fs::exists(backup_base_path)) + fs::create_directories(backup_base_path); + + auto disk = disk_it->second; + auto disk_object_storage = disk->createDiskObjectStorage(); + disk_object_storage->wrapWithBackup(name, backup_base_path); + + LOG_INFO( + &Poco::Logger::get("DiskBackup"), + "Registered backup disk (`{}`) with structure: {}", + name, assert_cast(disk_object_storage.get())->getStructure()); + + return disk_object_storage; + }; + + factory.registerDiskType("backup", creator); +} + +} diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 68cce9f2ccb6..ffcc376573a1 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -107,9 +107,9 @@ class CachedObjectStorage final : public IObjectStorage bool isWriteOnce() const override { return object_storage->isWriteOnce(); } - const std::string & getCacheConfigName() const { return cache_config_name; } + std::optional getLayerName() const override { return {cache_config_name}; } - ObjectStoragePtr getWrappedObjectStorage() { return object_storage; } + ObjectStoragePtr getWrappedObjectStorage() override { return object_storage; } bool supportParallelWrite() const override { return object_storage->supportParallelWrite(); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 931feeb22144..9419337a39bc 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #if ENABLE_DISTRIBUTED_CACHE @@ -714,6 +715,11 @@ String DiskObjectStorage::getWriteResourceNameNoLock() const return write_resource_name_from_config; } +void DiskObjectStorage::wrapWithBackup(const String & layer_name, const String & backup_base_path) +{ + object_storage = std::shared_ptr(new BackupObjectStorage(object_storage, backup_base_path, layer_name)); +} + std::unique_ptr DiskObjectStorage::readFile( const String & path, const ReadSettings & settings, diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 95fabdeb784f..af64c0e29bc4 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -219,14 +219,19 @@ friend class DiskObjectStorageReservation; return object_key_prefix; } + // Add a backup layer + void wrapWithBackup(const String & layer_name, const String & backup_base_path); + /// Add a cache layer. /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) /// There can be any number of cache layers: /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); - /// Get names of all cache layers. Name is how cache is defined in configuration file. - NameSet getCacheLayersNames() const override; + bool supportsLayers() const override { return true; } + + /// Get names of all backup and cache layers. Name is how backup/cache is defined in configuration file. + NameSet getLayersNames() const override; bool supportsStat() const override { return metadata_storage->supportsStat(); } struct stat stat(const String & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp b/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp index 9e5012dec54c..8fde4f469041 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp @@ -12,15 +12,18 @@ void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSetting object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); } -NameSet DiskObjectStorage::getCacheLayersNames() const +NameSet DiskObjectStorage::getLayersNames() const { NameSet cache_layers; auto current_object_storage = object_storage; - while (current_object_storage->supportsCache()) + while (current_object_storage != nullptr) { - auto * cached_object_storage = assert_cast(current_object_storage.get()); - cache_layers.insert(cached_object_storage->getCacheConfigName()); - current_object_storage = cached_object_storage->getWrappedObjectStorage(); + std::optional layer_name = current_object_storage->getLayerName(); + if (layer_name.has_value()) + { + cache_layers.insert(layer_name.value()); + } + current_object_storage = current_object_storage->getWrappedObjectStorage(); } return cache_layers; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 1f3a4278f135..b1c6389661c4 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -300,6 +300,10 @@ class IObjectStorage virtual bool isWriteOnce() const { return false; } virtual bool isPlain() const { return false; } + virtual std::optional getLayerName() const { return std::nullopt; } + + virtual ObjectStoragePtr getWrappedObjectStorage() { return nullptr; } + virtual bool supportParallelWrite() const { return false; } virtual ReadSettings patchSettings(const ReadSettings & read_settings) const; diff --git a/src/Disks/ReadOnlyDiskWrapper.h b/src/Disks/ReadOnlyDiskWrapper.h index cf00ade7af58..a081da710c3f 100644 --- a/src/Disks/ReadOnlyDiskWrapper.h +++ b/src/Disks/ReadOnlyDiskWrapper.h @@ -83,7 +83,8 @@ class ReadOnlyDiskWrapper : public IDisk DiskObjectStoragePtr createDiskObjectStorage() override { return delegate->createDiskObjectStorage(); } ObjectStoragePtr getObjectStorage() override { return delegate->getObjectStorage(); } - NameSet getCacheLayersNames() const override { return delegate->getCacheLayersNames(); } + bool supportsLayers() const override { return delegate->supportsLayers(); } + NameSet getLayersNames() const override { return delegate->getLayersNames(); } MetadataStoragePtr getMetadataStorage() override { return delegate->getMetadataStorage(); } diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 6c63e0771752..d969463a6315 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -13,6 +13,8 @@ void registerDiskLocal(DiskFactory & factory, bool global_skip_access_check); void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check); #endif +void registerDiskBackup(DiskFactory & factory, bool global_skip_access_check); + void registerDiskCache(DiskFactory & factory, bool global_skip_access_check); void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_check); @@ -27,6 +29,8 @@ void registerDisks(bool global_skip_access_check) registerDiskEncrypted(factory, global_skip_access_check); #endif + registerDiskBackup(factory, global_skip_access_check); + registerDiskCache(factory, global_skip_access_check); registerDiskObjectStorage(factory, global_skip_access_check); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3e88b576175a..473729bb766a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2154,18 +2154,26 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks, std::optionalgetDelegateDiskIfExists(); - if (delegate && disk->getPath() == delegate->getPath()) + if (delegate && disk->getPath() == delegate->getPath()) { defined_disk_names.insert(delegate->getName()); + if (delegate->supportsLayers()) + { + // See comment below, this also applies for the layers for the delegate disk. + auto caches = delegate->getLayersNames(); + defined_disk_names.insert(caches.begin(), caches.end()); + } + } - if (disk->supportsCache()) + if (disk->supportsLayers()) { /// As cache is implemented on object storage layer, not on disk level, e.g. /// we have such structure: /// DiskObjectStorage(CachedObjectStorage(...(CachedObjectStored(ObjectStorage)...))) /// and disk_ptr->getName() here is the name of last delegate - ObjectStorage. /// So now we need to add cache layers to defined disk names. - auto caches = disk->getCacheLayersNames(); - defined_disk_names.insert(caches.begin(), caches.end()); + // Same thing applies with backup layers. + auto caches = disk->getLayersNames(); + defined_disk_names.insert(caches.begin(), caches.end()); } } From 04054a031e249caeeb31db67dca85c7e406fd020 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Thu, 18 Dec 2025 11:38:12 +0100 Subject: [PATCH 28/49] Fix uncaught exception if S3 storage fails If an exception happened while copying data, then the finalize call was not done explicitly. The finalize was then be done while the out object is destructed and that led to a fatal error, killing the entire server. To fix that, catch the exception that might happen while copyData is running, run finalize there and rethrow. The rethrow is very important, the finalize after a failing copyData can also not-throw, in which case the initial error would be forgotten and ClickHouse would believe the copy succeeded. Co-authored-by: Kevin Michel --- src/Disks/IDisk.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 962c944fd2ab..e9e812a5decc 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -69,7 +69,16 @@ void IDisk::copyFile( /// NOLINT auto in = readFile(from_file_path, read_settings); auto out = to_disk.writeFile(to_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings); - copyData(*in, *out, cancellation_hook); + try + { + copyData(*in, *out, cancellation_hook); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + out->finalize(); + throw; + } out->finalize(); } From 6295f3bb92bc66c875bdd44575c1078d509506a8 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Thu, 18 Dec 2025 11:53:56 +0100 Subject: [PATCH 29/49] Skip attempt to create a container in azure blob storage When using Azure Blob Storage as a storage (table functions, external tables), the container is expected to already exist and is managed externally. Setting container_already_exists = true in StorageAzureConfiguration::createObjectStorage() optimizes initialization by skipping unnecessary container existence checks and creation attempts, eliminating 1-2 Azure API calls per storage initialization. This improves performance and avoids handling "container already exists" exceptions during initialization. The pattern matches the approach used in BackupIO_AzureBlobStorage. Co-authored-by: Tilman Moeller --- src/Storages/ObjectStorage/Azure/Configuration.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index c068a87d52c6..ab704b143b51 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -89,6 +89,7 @@ ObjectStoragePtr StorageAzureConfiguration::createObjectStorage(ContextPtr conte { assertInitialized(); + connection_params.endpoint.container_already_exists = true; auto settings = AzureBlobStorage::getRequestSettings(context->getSettingsRef()); auto client = AzureBlobStorage::getContainerClient(connection_params, is_readonly); From 4a4a477c98e43e699c30d2d553c1a397f1a252db Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 22 Dec 2025 13:50:49 +0100 Subject: [PATCH 30/49] Add Kafka configuration support for SASL and SSL settings This commit introduces new enums for Kafka SASL mechanisms, security protocols, and SSL endpoint identification algorithms. It also implements case-insensitive parsing for these settings to ensure backward compatibility. The KafkaConfigLoader is updated to handle these new configurations, allowing for more flexible and secure Kafka connections. Changes include: - Added `KafkaSASLMechanism`, `KafkaSecurityProtocol`, and `KafkaSSLEndpointIdentificationAlgorithm` enums. - Implemented `toString` and `fromString` methods for these enums with case-insensitive support for backwards compability. - Updated KafkaConfigLoader to apply security and SSL settings from the configuration parameters. Co-authored-by: Kevin Michel --- src/Core/SettingsEnums.cpp | 98 ++++++++++++++++++++++++ src/Core/SettingsEnums.h | 30 ++++++++ src/Storages/Kafka/KafkaConfigLoader.cpp | 59 ++++++++++---- src/Storages/Kafka/KafkaConfigLoader.h | 9 +++ src/Storages/Kafka/KafkaSettings.cpp | 8 +- src/Storages/Kafka/KafkaSettings.h | 3 + src/Storages/Kafka/StorageKafka.cpp | 28 ++++++- src/Storages/Kafka/StorageKafka2.cpp | 28 ++++++- 8 files changed, 244 insertions(+), 19 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 647828fee91e..82cccca2820f 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -164,6 +165,103 @@ IMPLEMENT_SETTING_ENUM(StreamingHandleErrorMode, ErrorCodes::BAD_ARGUMENTS, {"stream", StreamingHandleErrorMode::STREAM}, {"dead_letter_queue", StreamingHandleErrorMode::DEAD_LETTER_QUEUE}}) +// KafkaSASLMechanism: Manual implementation with case-insensitive parsing for backward compatibility +const String & SettingFieldKafkaSASLMechanismTraits::toString(KafkaSASLMechanism value) +{ + static const std::unordered_map map = { + {KafkaSASLMechanism::GSSAPI, "GSSAPI"}, + {KafkaSASLMechanism::PLAIN, "PLAIN"}, + {KafkaSASLMechanism::SCRAM_SHA_256, "SCRAM-SHA-256"}, + {KafkaSASLMechanism::SCRAM_SHA_512, "SCRAM-SHA-512"}, + {KafkaSASLMechanism::OAUTHBEARER, "OAUTHBEARER"} + }; + auto it = map.find(value); + if (it != map.end()) + return it->second; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of KafkaSASLMechanism: {}", static_cast(value)); +} + +KafkaSASLMechanism SettingFieldKafkaSASLMechanismTraits::fromString(std::string_view str) +{ + static const std::unordered_map map = { + {"GSSAPI", KafkaSASLMechanism::GSSAPI}, + {"PLAIN", KafkaSASLMechanism::PLAIN}, + {"SCRAM-SHA-256", KafkaSASLMechanism::SCRAM_SHA_256}, + {"SCRAM-SHA-512", KafkaSASLMechanism::SCRAM_SHA_512}, + {"OAUTHBEARER", KafkaSASLMechanism::OAUTHBEARER} + }; + String upper_str = Poco::toUpper(String{str}); + auto it = map.find(std::string_view{upper_str}); + if (it != map.end()) + return it->second; + String msg = "'GSSAPI', 'PLAIN', 'SCRAM-SHA-256', 'SCRAM-SHA-512', 'OAUTHBEARER'"; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of KafkaSASLMechanism: '{}'. Must be one of [{}]", String{str}, msg); +} + +// KafkaSecurityProtocol: Manual implementation with case-insensitive parsing for backward compatibility +const String & SettingFieldKafkaSecurityProtocolTraits::toString(KafkaSecurityProtocol value) +{ + static const std::unordered_map map = { + {KafkaSecurityProtocol::PLAINTEXT, "PLAINTEXT"}, + {KafkaSecurityProtocol::SSL, "SSL"}, + {KafkaSecurityProtocol::SASL_PLAINTEXT, "SASL_PLAINTEXT"}, + {KafkaSecurityProtocol::SASL_SSL, "SASL_SSL"} + }; + auto it = map.find(value); + if (it != map.end()) + return it->second; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of KafkaSecurityProtocol: {}", static_cast(value)); +} + +KafkaSecurityProtocol SettingFieldKafkaSecurityProtocolTraits::fromString(std::string_view str) +{ + static const std::unordered_map map = { + {"PLAINTEXT", KafkaSecurityProtocol::PLAINTEXT}, + {"SSL", KafkaSecurityProtocol::SSL}, + {"SASL_PLAINTEXT", KafkaSecurityProtocol::SASL_PLAINTEXT}, + {"SASL_SSL", KafkaSecurityProtocol::SASL_SSL} + }; + String upper_str = Poco::toUpper(String{str}); + // Handle underscore vs hyphen variations (sasl_plaintext vs sasl-plaintext) for backward compatibility + if (upper_str == "SASL-PLAINTEXT") + upper_str = "SASL_PLAINTEXT"; + else if (upper_str == "SASL-SSL") + upper_str = "SASL_SSL"; + auto it = map.find(std::string_view{upper_str}); + if (it != map.end()) + return it->second; + String msg = "'PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL'"; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of KafkaSecurityProtocol: '{}'. Must be one of [{}]", String{str}, msg); +} + +// KafkaSSLEndpointIdentificationAlgorithm: Manual implementation with case-insensitive parsing +const String & SettingFieldKafkaSSLEndpointIdentificationAlgorithmTraits::toString(KafkaSSLEndpointIdentificationAlgorithm value) +{ + static const std::unordered_map map = { + {KafkaSSLEndpointIdentificationAlgorithm::NONE, "none"}, + {KafkaSSLEndpointIdentificationAlgorithm::HTTPS, "https"} + }; + auto it = map.find(value); + if (it != map.end()) + return it->second; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of KafkaSSLEndpointIdentificationAlgorithm: {}", static_cast(value)); +} + +KafkaSSLEndpointIdentificationAlgorithm SettingFieldKafkaSSLEndpointIdentificationAlgorithmTraits::fromString(std::string_view str) +{ + static const std::unordered_map map = { + {"none", KafkaSSLEndpointIdentificationAlgorithm::NONE}, + {"https", KafkaSSLEndpointIdentificationAlgorithm::HTTPS} + }; + String lower_str = Poco::toLower(String{str}); // This enum uses lowercase values + auto it = map.find(std::string_view{lower_str}); + if (it != map.end()) + return it->second; + String msg = "'none', 'https'"; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of KafkaSSLEndpointIdentificationAlgorithm: '{}'. Must be one of [{}]", String{str}, msg); +} + + IMPLEMENT_SETTING_ENUM(ShortCircuitFunctionEvaluation, ErrorCodes::BAD_ARGUMENTS, {{"enable", ShortCircuitFunctionEvaluation::ENABLE}, {"force_enable", ShortCircuitFunctionEvaluation::FORCE_ENABLE}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 77b8b14737d3..310456ea6599 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -266,6 +266,36 @@ DECLARE_SETTING_ENUM(DistributedDDLOutputMode) DECLARE_SETTING_ENUM(StreamingHandleErrorMode) +enum class KafkaSASLMechanism +{ + GSSAPI = 0, + PLAIN, + SCRAM_SHA_256, + SCRAM_SHA_512, + OAUTHBEARER, +}; + +DECLARE_SETTING_ENUM(KafkaSASLMechanism) + +enum class KafkaSecurityProtocol +{ + PLAINTEXT = 0, + SSL, + SASL_PLAINTEXT, + SASL_SSL +}; + +DECLARE_SETTING_ENUM(KafkaSecurityProtocol) + +enum class KafkaSSLEndpointIdentificationAlgorithm +{ + NONE = 0, + HTTPS, +}; + +DECLARE_SETTING_ENUM(KafkaSSLEndpointIdentificationAlgorithm) + + DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) enum class TransactionsWaitCSNMode : uint8_t diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp index e4712c4d5f85..25cfbedd8177 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.cpp +++ b/src/Storages/Kafka/KafkaConfigLoader.cpp @@ -1,11 +1,13 @@ #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -28,10 +30,14 @@ namespace DB namespace KafkaSetting { - extern const KafkaSettingsString kafka_security_protocol; - extern const KafkaSettingsString kafka_sasl_mechanism; + extern const KafkaSettingsKafkaSecurityProtocol kafka_security_protocol; + extern const KafkaSettingsKafkaSASLMechanism kafka_sasl_mechanism; extern const KafkaSettingsString kafka_sasl_username; extern const KafkaSettingsString kafka_sasl_password; + extern const KafkaSettingsKafkaSSLEndpointIdentificationAlgorithm kafka_ssl_endpoint_identification_algorithm; + extern const KafkaSettingsString kafka_ssl_ca_location; + extern const KafkaSettingsString kafka_ssl_certificate_location; + extern const KafkaSettingsString kafka_ssl_key_location; } namespace ErrorCodes @@ -156,6 +162,22 @@ template struct KafkaInterceptors; namespace { +String toCppKafkaString(KafkaSecurityProtocol protocol) +{ + return Poco::toLower(SettingFieldKafkaSecurityProtocolTraits::toString(protocol)); +} + +String toCppKafkaString(KafkaSASLMechanism mechanism) +{ + // librdkafka expects uppercase values for sasl.mechanism (GSSAPI, PLAIN, SCRAM-SHA-256, etc.) + return SettingFieldKafkaSASLMechanismTraits::toString(mechanism); +} + +String toCppKafkaString(KafkaSSLEndpointIdentificationAlgorithm algorithm) +{ + return Poco::toLower(SettingFieldKafkaSSLEndpointIdentificationAlgorithmTraits::toString(algorithm)); +} + void setKafkaConfigValue(cppkafka::Configuration & kafka_config, const String & key, const String & value) { /// "log_level" has valid underscore, the remaining librdkafka setting use dot.separated.format which isn't acceptable for XML. @@ -341,24 +363,35 @@ void loadProducerConfig(cppkafka::Configuration & kafka_config, const KafkaConfi loadFromConfig(kafka_config, params, producer_path); } -template +template void updateGlobalConfiguration( cppkafka::Configuration & kafka_config, TKafkaStorage & storage, - const KafkaConfigLoader::LoadConfigParams & params, + const TParams & params, IKafkaExceptionInfoSinkWeakPtr exception_info_sink_ptr = IKafkaExceptionInfoSinkWeakPtr()) { loadFromConfig(kafka_config, params, KafkaConfigLoader::CONFIG_KAFKA_TAG); - auto kafka_settings = storage.getKafkaSettings(); - if (!kafka_settings[KafkaSetting::kafka_security_protocol].value.empty()) - kafka_config.set("security.protocol", kafka_settings[KafkaSetting::kafka_security_protocol]); - if (!kafka_settings[KafkaSetting::kafka_sasl_mechanism].value.empty()) - kafka_config.set("sasl.mechanism", kafka_settings[KafkaSetting::kafka_sasl_mechanism]); - if (!kafka_settings[KafkaSetting::kafka_sasl_username].value.empty()) - kafka_config.set("sasl.username", kafka_settings[KafkaSetting::kafka_sasl_username]); - if (!kafka_settings[KafkaSetting::kafka_sasl_password].value.empty()) - kafka_config.set("sasl.password", kafka_settings[KafkaSetting::kafka_sasl_password]); + // Apply security protocol + kafka_config.set("security.protocol", toCppKafkaString(params.security_protocol)); + + // Apply SASL mechanism + kafka_config.set("sasl.mechanism", toCppKafkaString(params.sasl_mechanism)); + + if (!params.sasl_username.empty()) + kafka_config.set("sasl.username", params.sasl_username); + if (!params.sasl_password.empty()) + kafka_config.set("sasl.password", params.sasl_password); + + // Apply SSL settings if provided + if (!params.ssl_ca_location.empty()) + kafka_config.set("ssl.ca.location", params.ssl_ca_location); + if (!params.ssl_certificate_location.empty()) + kafka_config.set("ssl.certificate.location", params.ssl_certificate_location); + if (!params.ssl_key_location.empty()) + kafka_config.set("ssl.key.location", params.ssl_key_location); + if (params.ssl_endpoint_identification_algorithm != KafkaSSLEndpointIdentificationAlgorithm::NONE) + kafka_config.set("ssl.endpoint.identification.algorithm", toCppKafkaString(params.ssl_endpoint_identification_algorithm)); #if USE_KRB5 if (kafka_config.has_property("sasl.kerberos.kinit.cmd")) diff --git a/src/Storages/Kafka/KafkaConfigLoader.h b/src/Storages/Kafka/KafkaConfigLoader.h index a007218723c5..52c9c03fe3b1 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.h +++ b/src/Storages/Kafka/KafkaConfigLoader.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -28,6 +29,14 @@ struct KafkaConfigLoader String & collection_name; const Names & topics; LoggerPtr & log; + KafkaSecurityProtocol security_protocol; + KafkaSASLMechanism sasl_mechanism; + String sasl_username; + String sasl_password; + KafkaSSLEndpointIdentificationAlgorithm ssl_endpoint_identification_algorithm; + String ssl_ca_location; + String ssl_certificate_location; + String ssl_key_location; }; struct ConsumerConfigParams : public LoadConfigParams diff --git a/src/Storages/Kafka/KafkaSettings.cpp b/src/Storages/Kafka/KafkaSettings.cpp index 35fa5f0e883f..af4be81ece9e 100644 --- a/src/Storages/Kafka/KafkaSettings.cpp +++ b/src/Storages/Kafka/KafkaSettings.cpp @@ -43,10 +43,14 @@ namespace ErrorCodes DECLARE(UInt64, kafka_max_rows_per_message, 1, "The maximum number of rows produced in one kafka message for row-based formats.", 0) \ DECLARE(String, kafka_keeper_path, "", "The path to the table in ClickHouse Keeper", 0) \ DECLARE(String, kafka_replica_name, "", "The replica name in ClickHouse Keeper", 0) \ - DECLARE(String, kafka_security_protocol, "", "Protocol used to communicate with brokers.", 0) \ - DECLARE(String, kafka_sasl_mechanism, "", "SASL mechanism to use for authentication. Supported: GSSAPI, PLAIN, SCRAM-SHA-256, SCRAM-SHA-512, OAUTHBEARER.", 0) \ + DECLARE(KafkaSecurityProtocol, kafka_security_protocol, KafkaSecurityProtocol::PLAINTEXT, "Protocol used to communicate with brokers. Possible values: PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL.", 0) \ + DECLARE(KafkaSASLMechanism, kafka_sasl_mechanism, KafkaSASLMechanism::GSSAPI, "SASL mechanism to use for authentication. Supported: GSSAPI, PLAIN, SCRAM-SHA-256, SCRAM-SHA-512, OAUTHBEARER.", 0) \ DECLARE(String, kafka_sasl_username, "", "SASL username for use with the PLAIN and SASL-SCRAM-.. mechanisms", 0) \ DECLARE(String, kafka_sasl_password, "", "SASL password for use with the PLAIN and SASL-SCRAM-.. mechanisms", 0) \ + DECLARE(KafkaSSLEndpointIdentificationAlgorithm, kafka_ssl_endpoint_identification_algorithm, KafkaSSLEndpointIdentificationAlgorithm::NONE, "SSL endpoint identification algorithm. Possible values: none, https.", 0) \ + DECLARE(String, kafka_ssl_ca_location, "", "CA certificate file path for SSL/TLS authentication.", 0) \ + DECLARE(String, kafka_ssl_certificate_location, "", "Client certificate file path for SSL/TLS authentication.", 0) \ + DECLARE(String, kafka_ssl_key_location, "", "Client private key file path for SSL/TLS authentication.", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index a7362c8ff44c..9106c3d09340 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -33,6 +33,9 @@ const auto KAFKA_CONSUMERS_POOL_TTL_MS_MAX = 600'000; M(CLASS_NAME, IdentifierQuotingStyle) \ M(CLASS_NAME, Int64) \ M(CLASS_NAME, IntervalOutputFormat) \ + M(CLASS_NAME, KafkaSASLMechanism) \ + M(CLASS_NAME, KafkaSecurityProtocol) \ + M(CLASS_NAME, KafkaSSLEndpointIdentificationAlgorithm) \ M(CLASS_NAME, Milliseconds) \ M(CLASS_NAME, MsgPackUUIDRepresentation) \ M(CLASS_NAME, ORCCompression) \ diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 71f4de7f4ca8..d590f11a84e5 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -88,6 +88,14 @@ namespace KafkaSetting extern const KafkaSettingsUInt64 kafka_poll_max_batch_size; extern const KafkaSettingsMilliseconds kafka_poll_timeout_ms; extern const KafkaSettingsString kafka_schema; + extern const KafkaSettingsKafkaSASLMechanism kafka_sasl_mechanism; + extern const KafkaSettingsString kafka_sasl_password; + extern const KafkaSettingsString kafka_sasl_username; + extern const KafkaSettingsKafkaSecurityProtocol kafka_security_protocol; + extern const KafkaSettingsString kafka_ssl_ca_location; + extern const KafkaSettingsString kafka_ssl_certificate_location; + extern const KafkaSettingsKafkaSSLEndpointIdentificationAlgorithm kafka_ssl_endpoint_identification_algorithm; + extern const KafkaSettingsString kafka_ssl_key_location; extern const KafkaSettingsBool kafka_thread_per_consumer; extern const KafkaSettingsString kafka_topic_list; } @@ -461,7 +469,15 @@ KafkaConsumerPtr StorageKafka::createKafkaConsumer(size_t consumer_number) cppkafka::Configuration StorageKafka::getConsumerConfiguration(size_t consumer_number, IKafkaExceptionInfoSinkPtr exception_info_sink_ptr) { KafkaConfigLoader::ConsumerConfigParams params{ - {getContext()->getConfigRef(), collection_name, topics, log}, + {getContext()->getConfigRef(), collection_name, topics, log, + (*kafka_settings)[KafkaSetting::kafka_security_protocol].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_mechanism].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_username].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_password].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_endpoint_identification_algorithm].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_ca_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_certificate_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_key_location].value}, brokers, group, num_consumers > 1, @@ -474,7 +490,15 @@ cppkafka::Configuration StorageKafka::getConsumerConfiguration(size_t consumer_n cppkafka::Configuration StorageKafka::getProducerConfiguration() { KafkaConfigLoader::ProducerConfigParams params{ - {getContext()->getConfigRef(), collection_name, topics, log}, + {getContext()->getConfigRef(), collection_name, topics, log, + (*kafka_settings)[KafkaSetting::kafka_security_protocol].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_mechanism].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_username].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_password].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_endpoint_identification_algorithm].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_ca_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_certificate_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_key_location].value}, brokers, client_id}; return KafkaConfigLoader::getProducerConfiguration(*this, params); diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index c55da9c24790..c084ddd415f4 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -104,6 +104,14 @@ namespace KafkaSetting extern const KafkaSettingsMilliseconds kafka_poll_timeout_ms; extern const KafkaSettingsString kafka_replica_name; extern const KafkaSettingsString kafka_schema; + extern const KafkaSettingsKafkaSASLMechanism kafka_sasl_mechanism; + extern const KafkaSettingsString kafka_sasl_password; + extern const KafkaSettingsString kafka_sasl_username; + extern const KafkaSettingsKafkaSecurityProtocol kafka_security_protocol; + extern const KafkaSettingsString kafka_ssl_ca_location; + extern const KafkaSettingsString kafka_ssl_certificate_location; + extern const KafkaSettingsKafkaSSLEndpointIdentificationAlgorithm kafka_ssl_endpoint_identification_algorithm; + extern const KafkaSettingsString kafka_ssl_key_location; extern const KafkaSettingsBool kafka_thread_per_consumer; extern const KafkaSettingsString kafka_topic_list; } @@ -457,7 +465,15 @@ KafkaConsumer2Ptr StorageKafka2::createKafkaConsumer(size_t consumer_number) cppkafka::Configuration StorageKafka2::getConsumerConfiguration(size_t consumer_number, IKafkaExceptionInfoSinkPtr exception_sink) { KafkaConfigLoader::ConsumerConfigParams params{ - {getContext()->getConfigRef(), collection_name, topics, log}, + {getContext()->getConfigRef(), collection_name, topics, log, + (*kafka_settings)[KafkaSetting::kafka_security_protocol].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_mechanism].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_username].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_password].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_endpoint_identification_algorithm].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_ca_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_certificate_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_key_location].value}, brokers, group, num_consumers > 1, @@ -475,7 +491,15 @@ cppkafka::Configuration StorageKafka2::getConsumerConfiguration(size_t consumer_ cppkafka::Configuration StorageKafka2::getProducerConfiguration() { KafkaConfigLoader::ProducerConfigParams params{ - {getContext()->getConfigRef(), collection_name, topics, log}, + {getContext()->getConfigRef(), collection_name, topics, log, + (*kafka_settings)[KafkaSetting::kafka_security_protocol].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_mechanism].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_username].value, + (*kafka_settings)[KafkaSetting::kafka_sasl_password].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_endpoint_identification_algorithm].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_ca_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_certificate_location].value, + (*kafka_settings)[KafkaSetting::kafka_ssl_key_location].value}, brokers, client_id}; return KafkaConfigLoader::getProducerConfiguration(*this, params); From 44983c11597b8e883f75717f8d0815af392a22db Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 22 Dec 2025 15:34:01 +0100 Subject: [PATCH 31/49] Allow decreasing number of Kafka consumers to zero Decrease the lower bound on the number of Kafka consumers. This allows setting it to zero, effectively disabling Kafka consumers globally without relying on SQL statement execution, which would allow users to interfere adversely with management operations. Changes: - Remove validation in StorageKafkaUtils.cpp that prevented num_consumers < 1, allowing kafka_num_consumers to be set to 0 - Fix division by zero in StorageKafka::getMaxBlockSize() when num_consumers is 0 by using a guard value of 1 - Fix division by zero in StorageKafka2::getMaxBlockSize() with the same guard (this fix was missing from the original patch) When num_consumers is set to 0: - Consumer loops safely skip execution (for i < 0 doesn't execute) - Container operations (resize, reserve) handle 0 correctly - Block size calculation uses guard value to prevent division by zero --- src/Storages/Kafka/StorageKafka.cpp | 3 ++- src/Storages/Kafka/StorageKafka2.cpp | 3 ++- src/Storages/Kafka/StorageKafkaUtils.cpp | 4 ---- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index d590f11a84e5..dcb0e7f3bf57 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -558,8 +558,9 @@ void StorageKafka::cleanConsumersByTTL() size_t StorageKafka::getMaxBlockSize() const { + size_t nonzero_num_consumers = num_consumers > 0 ? num_consumers : 1; // prevent division by zero return (*kafka_settings)[KafkaSetting::kafka_max_block_size].changed ? (*kafka_settings)[KafkaSetting::kafka_max_block_size].value - : (getContext()->getSettingsRef()[Setting::max_insert_block_size].value / num_consumers); + : (getContext()->getSettingsRef()[Setting::max_insert_block_size].value / nonzero_num_consumers); } size_t StorageKafka::getPollMaxBatchSize() const diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index c084ddd415f4..69f863faa1c6 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -507,8 +507,9 @@ cppkafka::Configuration StorageKafka2::getProducerConfiguration() size_t StorageKafka2::getMaxBlockSize() const { + size_t nonzero_num_consumers = num_consumers > 0 ? num_consumers : 1; // prevent division by zero return (*kafka_settings)[KafkaSetting::kafka_max_block_size].changed ? (*kafka_settings)[KafkaSetting::kafka_max_block_size].value - : (getContext()->getSettingsRef()[Setting::max_insert_block_size].value / num_consumers); + : (getContext()->getSettingsRef()[Setting::max_insert_block_size].value / nonzero_num_consumers); } size_t StorageKafka2::getPollMaxBatchSize() const diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp index 38439cda5866..30e5b6309e7e 100644 --- a/src/Storages/Kafka/StorageKafkaUtils.cpp +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -205,10 +205,6 @@ void registerStorageKafka(StorageFactory & factory) "See also https://clickhouse.com/docs/integrations/kafka/kafka-table-engine#tuning-performance", max_consumers); } - if (num_consumers < 1) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of consumers can not be lower than 1"); - } if ((*kafka_settings)[KafkaSetting::kafka_max_block_size].changed && (*kafka_settings)[KafkaSetting::kafka_max_block_size].value < 1) { From 27b27d7b7bec018399b5ee851c437840472e44ab Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 22 Dec 2025 17:10:43 +0100 Subject: [PATCH 32/49] Support per-table schema registry with authentication Add an extra option for tables with the Kafka engine to pass the URL of a schema registry. Also add support for schema registry basic authentication via username:password in the URL. Changes: - Add kafka_format_avro_schema_registry_url setting to KafkaSettings - Store format_avro_schema_registry_url in StorageKafka class - Propagate schema registry URL to format settings via createSettingsAdjustments() (works for both StorageKafka and StorageKafka2) - Add basic authentication parsing in AvroRowInputFormat::SchemaRegistry to extract username:password from URL and authenticate HTTP requests This allows per-table configuration of Avro Schema Registry URLs with authentication, enabling different Kafka tables to use different schema registries or credentials. Usage example: CREATE TABLE kafka_table (...) ENGINE = Kafka(...) SETTINGS kafka_format_avro_schema_registry_url = 'http://user:pass@registry:8081'; Co-authored-by: Kevin Michel --- .../Formats/Impl/AvroRowInputFormat.cpp | 20 ++++++++++++++++++- src/Storages/Kafka/KafkaSettings.cpp | 1 + src/Storages/Kafka/StorageKafka.cpp | 2 ++ src/Storages/Kafka/StorageKafka.h | 1 + src/Storages/Kafka/StorageKafkaUtils.cpp | 6 ++++++ 5 files changed, 29 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 1c67aa2f66d4..f7cf7f2ef4b6 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1074,7 +1074,10 @@ class AvroConfluentRowInputFormat::SchemaRegistry try { Poco::URI url(base_url, base_url.getPath() + "/schemas/ids/" + std::to_string(id)); - LOG_TRACE((getLogger("AvroConfluentRowInputFormat")), "Fetching schema id = {} from url {}", id, url.toString()); + // Create sanitized URL for logging (without credentials) + Poco::URI sanitized_url(url); + sanitized_url.setUserInfo(""); + LOG_TRACE((getLogger("AvroConfluentRowInputFormat")), "Fetching schema id = {} from url {}", id, sanitized_url.toString()); /// One second for connect/send/receive. Just in case. auto timeouts = ConnectionTimeouts() @@ -1083,6 +1086,21 @@ class AvroConfluentRowInputFormat::SchemaRegistry .withReceiveTimeout(1); Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + const auto & user_info = base_url.getUserInfo(); + if (!user_info.empty()) + { + std::size_t n = user_info.find(':'); + if (n != std::string::npos) + { + Poco::Net::HTTPBasicCredentials credentials; + credentials.setUsername(user_info.substr(0, n)); + credentials.setPassword(user_info.substr(n + 1)); + if (!credentials.getUsername().empty()) + { + credentials.authenticate(request); + } + } + } if (url.getPort()) request.setHost(url.getHost(), url.getPort()); else diff --git a/src/Storages/Kafka/KafkaSettings.cpp b/src/Storages/Kafka/KafkaSettings.cpp index af4be81ece9e..f272da213dfb 100644 --- a/src/Storages/Kafka/KafkaSettings.cpp +++ b/src/Storages/Kafka/KafkaSettings.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes DECLARE(String, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \ /* those are mapped to format factory settings */ \ DECLARE(String, kafka_format, "", "The message format for Kafka engine.", 0) \ + DECLARE(String, kafka_format_avro_schema_registry_url, "", "For AvroConfluent format: Kafka Schema Registry URL.", 0) \ DECLARE(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ DECLARE(UInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ /* default is = max_insert_block_size / kafka_num_consumers */ \ diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index dcb0e7f3bf57..1d22bb01b672 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -98,6 +98,7 @@ namespace KafkaSetting extern const KafkaSettingsString kafka_ssl_key_location; extern const KafkaSettingsBool kafka_thread_per_consumer; extern const KafkaSettingsString kafka_topic_list; + extern const KafkaSettingsString kafka_format_avro_schema_registry_url; } namespace ErrorCodes @@ -188,6 +189,7 @@ StorageKafka::StorageKafka( ? StorageKafkaUtils::getDefaultClientId(table_id_) : getContext()->getMacros()->expand((*kafka_settings)[KafkaSetting::kafka_client_id].value, macros_info)) , format_name(getContext()->getMacros()->expand((*kafka_settings)[KafkaSetting::kafka_format].value)) + , format_avro_schema_registry_url((*kafka_settings)[KafkaSetting::kafka_format_avro_schema_registry_url].value) , max_rows_per_message((*kafka_settings)[KafkaSetting::kafka_max_rows_per_message].value) , schema_name(getContext()->getMacros()->expand((*kafka_settings)[KafkaSetting::kafka_schema].value, macros_info)) , num_consumers((*kafka_settings)[KafkaSetting::kafka_num_consumers].value) diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index f2a47211f148..e1ac4a27a269 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -107,6 +107,7 @@ class StorageKafka final : public IStorage, WithContext const String group; const String client_id; const String format_name; + const String format_avro_schema_registry_url; const size_t max_rows_per_message; const String schema_name; const size_t num_consumers; /// total number of consumers diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp index 30e5b6309e7e..e5bf7364e6c1 100644 --- a/src/Storages/Kafka/StorageKafkaUtils.cpp +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -82,6 +82,7 @@ namespace KafkaSetting extern const KafkaSettingsUInt64 kafka_skip_broken_messages; extern const KafkaSettingsBool kafka_thread_per_consumer; extern const KafkaSettingsString kafka_topic_list; + extern const KafkaSettingsString kafka_format_avro_schema_registry_url; } using namespace std::chrono_literals; @@ -520,6 +521,11 @@ SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const result.setSetting("input_format_csv_detect_header", false); result.setSetting("input_format_tsv_detect_header", false); result.setSetting("input_format_custom_detect_header", false); + const String & format_avro_schema_registry_url = kafka_settings[KafkaSetting::kafka_format_avro_schema_registry_url].value; + if (!format_avro_schema_registry_url.empty()) + { + result.emplace_back("format_avro_schema_registry_url", format_avro_schema_registry_url); + } return result; } From 5047b86c5581fb6637d0f55c1d807606469dcb56 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 23 Dec 2025 12:36:49 +0100 Subject: [PATCH 33/49] Add kafka_auto_offset_reset and kafka_date_time_input_format settings kafka_auto_offset_reset was previously hardcoded to "earliest" in the Kafka consumer configuration. This commit makes it configurable per table, allowing users to choose from: smallest, earliest, beginning, largest, latest, or end. kafka_date_time_input_format was only configurable through the global config file or the context (but context settings affecting CREATE query do not work well with replication/restoration). This commit makes it configurable per table as a Kafka table setting. The default value for kafka_auto_offset_reset is EARLIEST, which maintains backward compatibility with the previous hardcoded behavior. Co-authored-by: Kevin Michel --- src/Core/SettingsEnums.cpp | 7 +++++++ src/Core/SettingsEnums.h | 12 ++++++++++++ src/Storages/Kafka/KafkaConfigLoader.cpp | 2 +- src/Storages/Kafka/KafkaConfigLoader.h | 1 + src/Storages/Kafka/KafkaSettings.cpp | 2 ++ src/Storages/Kafka/KafkaSettings.h | 1 + src/Storages/Kafka/StorageKafka.cpp | 5 ++++- src/Storages/Kafka/StorageKafka2.cpp | 5 ++++- src/Storages/Kafka/StorageKafkaUtils.cpp | 3 +++ 9 files changed, 35 insertions(+), 3 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 82cccca2820f..2d97f901c8b4 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -261,6 +261,13 @@ KafkaSSLEndpointIdentificationAlgorithm SettingFieldKafkaSSLEndpointIdentificati throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected value of KafkaSSLEndpointIdentificationAlgorithm: '{}'. Must be one of [{}]", String{str}, msg); } +IMPLEMENT_SETTING_ENUM(KafkaAutoOffsetReset, ErrorCodes::BAD_ARGUMENTS, + {{"smallest", KafkaAutoOffsetReset::SMALLEST}, + {"earliest", KafkaAutoOffsetReset::EARLIEST}, + {"beginning", KafkaAutoOffsetReset::BEGINNING}, + {"largest", KafkaAutoOffsetReset::LARGEST}, + {"latest", KafkaAutoOffsetReset::LATEST}, + {"end", KafkaAutoOffsetReset::END}}) IMPLEMENT_SETTING_ENUM(ShortCircuitFunctionEvaluation, ErrorCodes::BAD_ARGUMENTS, {{"enable", ShortCircuitFunctionEvaluation::ENABLE}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 310456ea6599..6ad3529fc873 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -295,6 +295,18 @@ enum class KafkaSSLEndpointIdentificationAlgorithm DECLARE_SETTING_ENUM(KafkaSSLEndpointIdentificationAlgorithm) +enum class KafkaAutoOffsetReset +{ + SMALLEST = 0, // Automatically reset the offset to the smallest offset + EARLIEST, + BEGINNING, + LARGEST, // Automatically reset the offset to the largest offset + LATEST, + END, +}; + +DECLARE_SETTING_ENUM(KafkaAutoOffsetReset) + DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp index 25cfbedd8177..2cb3176e6dbf 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.cpp +++ b/src/Storages/Kafka/KafkaConfigLoader.cpp @@ -491,7 +491,7 @@ cppkafka::Configuration KafkaConfigLoader::getConsumerConfiguration(TKafkaStorag conf.set("client.id", params.client_id); conf.set("client.software.name", VERSION_NAME); conf.set("client.software.version", VERSION_DESCRIBE); - conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start + conf.set("auto.offset.reset", params.auto_offset_reset); // that allows to prevent fast draining of the librdkafka queue // during building of single insert block. Improves performance diff --git a/src/Storages/Kafka/KafkaConfigLoader.h b/src/Storages/Kafka/KafkaConfigLoader.h index 52c9c03fe3b1..61b1e6ff2845 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.h +++ b/src/Storages/Kafka/KafkaConfigLoader.h @@ -47,6 +47,7 @@ struct KafkaConfigLoader size_t consumer_number; String client_id; size_t max_block_size; + String auto_offset_reset; }; struct ProducerConfigParams : public LoadConfigParams diff --git a/src/Storages/Kafka/KafkaSettings.cpp b/src/Storages/Kafka/KafkaSettings.cpp index f272da213dfb..cd430c1af48b 100644 --- a/src/Storages/Kafka/KafkaSettings.cpp +++ b/src/Storages/Kafka/KafkaSettings.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes /* those are mapped to format factory settings */ \ DECLARE(String, kafka_format, "", "The message format for Kafka engine.", 0) \ DECLARE(String, kafka_format_avro_schema_registry_url, "", "For AvroConfluent format: Kafka Schema Registry URL.", 0) \ + DECLARE(DateTimeInputFormat, kafka_date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Date and time input format for Kafka engine.", 0) \ DECLARE(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ DECLARE(UInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ /* default is = max_insert_block_size / kafka_num_consumers */ \ @@ -52,6 +53,7 @@ namespace ErrorCodes DECLARE(String, kafka_ssl_ca_location, "", "CA certificate file path for SSL/TLS authentication.", 0) \ DECLARE(String, kafka_ssl_certificate_location, "", "Client certificate file path for SSL/TLS authentication.", 0) \ DECLARE(String, kafka_ssl_key_location, "", "Client private key file path for SSL/TLS authentication.", 0) \ + DECLARE(KafkaAutoOffsetReset, kafka_auto_offset_reset, KafkaAutoOffsetReset::EARLIEST, "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server. Possible values: smallest, earliest, beginning, largest, latest, end.", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 9106c3d09340..0a61572082ea 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -33,6 +33,7 @@ const auto KAFKA_CONSUMERS_POOL_TTL_MS_MAX = 600'000; M(CLASS_NAME, IdentifierQuotingStyle) \ M(CLASS_NAME, Int64) \ M(CLASS_NAME, IntervalOutputFormat) \ + M(CLASS_NAME, KafkaAutoOffsetReset) \ M(CLASS_NAME, KafkaSASLMechanism) \ M(CLASS_NAME, KafkaSecurityProtocol) \ M(CLASS_NAME, KafkaSSLEndpointIdentificationAlgorithm) \ diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 1d22bb01b672..64db1ce951cc 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -39,6 +39,7 @@ #include #include +#include #include #include @@ -91,6 +92,7 @@ namespace KafkaSetting extern const KafkaSettingsKafkaSASLMechanism kafka_sasl_mechanism; extern const KafkaSettingsString kafka_sasl_password; extern const KafkaSettingsString kafka_sasl_username; + extern const KafkaSettingsKafkaAutoOffsetReset kafka_auto_offset_reset; extern const KafkaSettingsKafkaSecurityProtocol kafka_security_protocol; extern const KafkaSettingsString kafka_ssl_ca_location; extern const KafkaSettingsString kafka_ssl_certificate_location; @@ -485,7 +487,8 @@ cppkafka::Configuration StorageKafka::getConsumerConfiguration(size_t consumer_n num_consumers > 1, consumer_number, client_id, - getMaxBlockSize()}; + getMaxBlockSize(), + SettingFieldKafkaAutoOffsetResetTraits::toString((*kafka_settings)[KafkaSetting::kafka_auto_offset_reset].value)}; return KafkaConfigLoader::getConsumerConfiguration(*this, params, exception_info_sink_ptr); } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 69f863faa1c6..3a0a89a8e700 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -107,6 +108,7 @@ namespace KafkaSetting extern const KafkaSettingsKafkaSASLMechanism kafka_sasl_mechanism; extern const KafkaSettingsString kafka_sasl_password; extern const KafkaSettingsString kafka_sasl_username; + extern const KafkaSettingsKafkaAutoOffsetReset kafka_auto_offset_reset; extern const KafkaSettingsKafkaSecurityProtocol kafka_security_protocol; extern const KafkaSettingsString kafka_ssl_ca_location; extern const KafkaSettingsString kafka_ssl_certificate_location; @@ -479,7 +481,8 @@ cppkafka::Configuration StorageKafka2::getConsumerConfiguration(size_t consumer_ num_consumers > 1, consumer_number, client_id, - getMaxBlockSize()}; + getMaxBlockSize(), + SettingFieldKafkaAutoOffsetResetTraits::toString((*kafka_settings)[KafkaSetting::kafka_auto_offset_reset].value)}; auto kafka_config = KafkaConfigLoader::getConsumerConfiguration(*this, params, std::move(exception_sink)); // It is disabled, because in case of no materialized views are attached, it can cause live memory leak. To enable it, a similar cleanup mechanism must be introduced as for StorageKafka. kafka_config.set("statistics.interval.ms", "0"); diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp index e5bf7364e6c1..916dc6fa35ac 100644 --- a/src/Storages/Kafka/StorageKafkaUtils.cpp +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -83,6 +83,7 @@ namespace KafkaSetting extern const KafkaSettingsBool kafka_thread_per_consumer; extern const KafkaSettingsString kafka_topic_list; extern const KafkaSettingsString kafka_format_avro_schema_registry_url; + extern const KafkaSettingsDateTimeInputFormat kafka_date_time_input_format; } using namespace std::chrono_literals; @@ -527,6 +528,8 @@ SettingsChanges createSettingsAdjustments(KafkaSettings & kafka_settings, const result.emplace_back("format_avro_schema_registry_url", format_avro_schema_registry_url); } + result.emplace_back("date_time_input_format", kafka_settings[KafkaSetting::kafka_date_time_input_format].toString()); + return result; } From e89bc6303ab04cf760bafcfdde976b1afd3666aa Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 23 Dec 2025 14:53:17 +0100 Subject: [PATCH 34/49] Add extra settings to Kafka Table Engine These settings can be tweaked to help improve throughput when writing to Kafka using the Kafka table engine. The settings are configurable per table, allowing fine-grained control over producer behavior. New producer settings added: - kafka_producer_batch_size: Maximum size of a batch in bytes - kafka_producer_batch_num_messages: Maximum number of messages in a batch - kafka_producer_compression_codec: Compression codec (none, gzip, snappy, lz4, zstd) - kafka_producer_compression_level: Compression level (-1 for default) - kafka_producer_linger_ms: Delay to wait for messages to form batches - kafka_producer_queue_buffering_max_messages: Max messages in producer queue - kafka_producer_queue_buffering_max_kbytes: Max KB in producer queue - kafka_producer_request_required_acks: Required acknowledgments (-1, 0, or 1) All settings default to 0 or -1, which means librdkafka will use its default values. This maintains backward compatibility with existing configurations. Changes: - Added KafkaCompressionCodec enum with case-preserving string conversion - Extended ProducerConfigParams with all producer settings - Applied producer settings to cppkafka configuration in getProducerConfiguration() - Updated both StorageKafka and StorageKafka2 to pass settings to ProducerConfigParams --- src/Core/SettingsEnums.cpp | 7 +++++++ src/Core/SettingsEnums.h | 11 +++++++++++ src/Storages/Kafka/KafkaConfigLoader.cpp | 9 +++++++++ src/Storages/Kafka/KafkaConfigLoader.h | 8 ++++++++ src/Storages/Kafka/KafkaSettings.cpp | 8 ++++++++ src/Storages/Kafka/KafkaSettings.h | 1 + src/Storages/Kafka/StorageKafka.cpp | 18 +++++++++++++++++- src/Storages/Kafka/StorageKafka2.cpp | 18 +++++++++++++++++- 8 files changed, 78 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 2d97f901c8b4..db926fc89f1d 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -425,6 +425,13 @@ IMPLEMENT_SETTING_ENUM( {"glue", DatabaseDataLakeCatalogType::GLUE}, {"hive", DatabaseDataLakeCatalogType::ICEBERG_HIVE}}) +IMPLEMENT_SETTING_ENUM(KafkaCompressionCodec, ErrorCodes::BAD_ARGUMENTS, + {{"none", KafkaCompressionCodec::none}, + {"gzip", KafkaCompressionCodec::gzip}, + {"snappy", KafkaCompressionCodec::snappy}, + {"lz4", KafkaCompressionCodec::lz4}, + {"zstd", KafkaCompressionCodec::zstd}}) + IMPLEMENT_SETTING_ENUM( FileCachePolicy, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 6ad3529fc873..6828f1533ccd 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -414,6 +414,17 @@ DECLARE_SETTING_ENUM(ParallelReplicasMode) DECLARE_SETTING_ENUM(LocalFSReadMethod) +enum class KafkaCompressionCodec +{ + none, + gzip, + snappy, + lz4, + zstd, +}; + +DECLARE_SETTING_ENUM(KafkaCompressionCodec) + enum class ObjectStorageQueueMode : uint8_t { ORDERED, diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp index 2cb3176e6dbf..2f3131f6873e 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.cpp +++ b/src/Storages/Kafka/KafkaConfigLoader.cpp @@ -531,6 +531,15 @@ cppkafka::Configuration KafkaConfigLoader::getProducerConfiguration(TKafkaStorag conf.set("client.software.name", VERSION_NAME); conf.set("client.software.version", VERSION_DESCRIBE); + conf.set("batch.size", params.batch_size); + conf.set("batch.num.messages", params.batch_num_messages); + conf.set("compression.codec", params.compression_codec); + conf.set("compression.level", params.compression_level); + conf.set("linger.ms", params.linger_ms); + conf.set("queue.buffering.max.messages", params.queue_buffering_max_messages); + conf.set("queue.buffering.max.kbytes", params.queue_buffering_max_kbytes); + conf.set("request.required.acks", params.request_required_acks); + updateGlobalConfiguration(conf, storage, params); loadProducerConfig(conf, params); diff --git a/src/Storages/Kafka/KafkaConfigLoader.h b/src/Storages/Kafka/KafkaConfigLoader.h index 61b1e6ff2845..775a18c2445b 100644 --- a/src/Storages/Kafka/KafkaConfigLoader.h +++ b/src/Storages/Kafka/KafkaConfigLoader.h @@ -54,6 +54,14 @@ struct KafkaConfigLoader { String brokers; String client_id; + UInt64 batch_size; + UInt64 batch_num_messages; + String compression_codec; + Int64 compression_level; + UInt64 linger_ms; + UInt64 queue_buffering_max_messages; + UInt64 queue_buffering_max_kbytes; + Int64 request_required_acks; }; template diff --git a/src/Storages/Kafka/KafkaSettings.cpp b/src/Storages/Kafka/KafkaSettings.cpp index cd430c1af48b..79be9e6ee85b 100644 --- a/src/Storages/Kafka/KafkaSettings.cpp +++ b/src/Storages/Kafka/KafkaSettings.cpp @@ -54,6 +54,14 @@ namespace ErrorCodes DECLARE(String, kafka_ssl_certificate_location, "", "Client certificate file path for SSL/TLS authentication.", 0) \ DECLARE(String, kafka_ssl_key_location, "", "Client private key file path for SSL/TLS authentication.", 0) \ DECLARE(KafkaAutoOffsetReset, kafka_auto_offset_reset, KafkaAutoOffsetReset::EARLIEST, "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server. Possible values: smallest, earliest, beginning, largest, latest, end.", 0) \ + DECLARE(UInt64, kafka_producer_batch_size, 0, "Maximum size of a batch in bytes for Kafka producer.", 0) \ + DECLARE(UInt64, kafka_producer_batch_num_messages, 0, "Maximum number of messages in a batch for Kafka producer.", 0) \ + DECLARE(KafkaCompressionCodec, kafka_producer_compression_codec, KafkaCompressionCodec::none, "Compression codec for Kafka producer. Possible values: none, gzip, snappy, lz4, zstd.", 0) \ + DECLARE(Int64, kafka_producer_compression_level, -1, "Compression level for Kafka producer. -1 means use default compression level for the codec.", 0) \ + DECLARE(UInt64, kafka_producer_linger_ms, 0, "Delay in milliseconds to wait for messages in the producer queue to form batches.", 0) \ + DECLARE(UInt64, kafka_producer_queue_buffering_max_messages, 0, "Maximum number of messages allowed in the producer queue.", 0) \ + DECLARE(UInt64, kafka_producer_queue_buffering_max_kbytes, 0, "Maximum total size of messages allowed in the producer queue in kilobytes.", 0) \ + DECLARE(Int64, kafka_producer_request_required_acks, -1, "Number of acknowledgments required from brokers. -1 means wait for all replicas, 0 means no acknowledgment, 1 means wait for leader only.", 0) \ #define OBSOLETE_KAFKA_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Char, kafka_row_delimiter, '\0') \ diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 0a61572082ea..075c4497eb53 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -34,6 +34,7 @@ const auto KAFKA_CONSUMERS_POOL_TTL_MS_MAX = 600'000; M(CLASS_NAME, Int64) \ M(CLASS_NAME, IntervalOutputFormat) \ M(CLASS_NAME, KafkaAutoOffsetReset) \ + M(CLASS_NAME, KafkaCompressionCodec) \ M(CLASS_NAME, KafkaSASLMechanism) \ M(CLASS_NAME, KafkaSecurityProtocol) \ M(CLASS_NAME, KafkaSSLEndpointIdentificationAlgorithm) \ diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 64db1ce951cc..2af7f959d411 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -101,6 +101,14 @@ namespace KafkaSetting extern const KafkaSettingsBool kafka_thread_per_consumer; extern const KafkaSettingsString kafka_topic_list; extern const KafkaSettingsString kafka_format_avro_schema_registry_url; + extern const KafkaSettingsKafkaCompressionCodec kafka_producer_compression_codec; + extern const KafkaSettingsInt64 kafka_producer_compression_level; + extern const KafkaSettingsUInt64 kafka_producer_linger_ms; + extern const KafkaSettingsUInt64 kafka_producer_queue_buffering_max_messages; + extern const KafkaSettingsUInt64 kafka_producer_batch_size; + extern const KafkaSettingsUInt64 kafka_producer_batch_num_messages; + extern const KafkaSettingsInt64 kafka_producer_request_required_acks; + extern const KafkaSettingsUInt64 kafka_producer_queue_buffering_max_kbytes; } namespace ErrorCodes @@ -505,7 +513,15 @@ cppkafka::Configuration StorageKafka::getProducerConfiguration() (*kafka_settings)[KafkaSetting::kafka_ssl_certificate_location].value, (*kafka_settings)[KafkaSetting::kafka_ssl_key_location].value}, brokers, - client_id}; + client_id, + (*kafka_settings)[KafkaSetting::kafka_producer_batch_size].value, + (*kafka_settings)[KafkaSetting::kafka_producer_batch_num_messages].value, + SettingFieldKafkaCompressionCodecTraits::toString((*kafka_settings)[KafkaSetting::kafka_producer_compression_codec].value), + (*kafka_settings)[KafkaSetting::kafka_producer_compression_level].value, + (*kafka_settings)[KafkaSetting::kafka_producer_linger_ms].value, + (*kafka_settings)[KafkaSetting::kafka_producer_queue_buffering_max_messages].value, + (*kafka_settings)[KafkaSetting::kafka_producer_queue_buffering_max_kbytes].value, + (*kafka_settings)[KafkaSetting::kafka_producer_request_required_acks].value}; return KafkaConfigLoader::getProducerConfiguration(*this, params); } diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 3a0a89a8e700..660543a3b277 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -116,6 +116,14 @@ namespace KafkaSetting extern const KafkaSettingsString kafka_ssl_key_location; extern const KafkaSettingsBool kafka_thread_per_consumer; extern const KafkaSettingsString kafka_topic_list; + extern const KafkaSettingsKafkaCompressionCodec kafka_producer_compression_codec; + extern const KafkaSettingsInt64 kafka_producer_compression_level; + extern const KafkaSettingsUInt64 kafka_producer_linger_ms; + extern const KafkaSettingsUInt64 kafka_producer_queue_buffering_max_messages; + extern const KafkaSettingsUInt64 kafka_producer_batch_size; + extern const KafkaSettingsUInt64 kafka_producer_batch_num_messages; + extern const KafkaSettingsInt64 kafka_producer_request_required_acks; + extern const KafkaSettingsUInt64 kafka_producer_queue_buffering_max_kbytes; } namespace fs = std::filesystem; @@ -504,7 +512,15 @@ cppkafka::Configuration StorageKafka2::getProducerConfiguration() (*kafka_settings)[KafkaSetting::kafka_ssl_certificate_location].value, (*kafka_settings)[KafkaSetting::kafka_ssl_key_location].value}, brokers, - client_id}; + client_id, + (*kafka_settings)[KafkaSetting::kafka_producer_batch_size].value, + (*kafka_settings)[KafkaSetting::kafka_producer_batch_num_messages].value, + SettingFieldKafkaCompressionCodecTraits::toString((*kafka_settings)[KafkaSetting::kafka_producer_compression_codec].value), + (*kafka_settings)[KafkaSetting::kafka_producer_compression_level].value, + (*kafka_settings)[KafkaSetting::kafka_producer_linger_ms].value, + (*kafka_settings)[KafkaSetting::kafka_producer_queue_buffering_max_messages].value, + (*kafka_settings)[KafkaSetting::kafka_producer_queue_buffering_max_kbytes].value, + (*kafka_settings)[KafkaSetting::kafka_producer_request_required_acks].value}; return KafkaConfigLoader::getProducerConfiguration(*this, params); } From 51de03dd3b52953bcb73716018f15f0fe770811f Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Fri, 2 Jan 2026 11:34:04 +0100 Subject: [PATCH 35/49] Unlock PostgreSQL database Fixes an issue where the single mutex in each database using the PostgreSQL engine is locked while attempting to connect to PostgreSQL and while running queries. This is a problem because the same lock is held while reading the database name and its comment field. This means that a single failing service integration could make it almost impossible to read the `system.databases` metadata table, which is used for monitoring and other critical operations. This would also make it really hard to drop the problematic integration database, since the drop operation also does PostgreSQL queries. The `cached` option of the PostgreSQL database engine does not improve the situation: It only caches the table structure but not the table existence, this is why drop/attach/detach/list operations all need PostgreSQL queries. Changes: - Converted helper functions (getTableNameForLogs, formatTableName, checkPostgresTable, fetchTable) to static/free functions that don't require class access - Refactored all methods to copy configuration values while holding the lock briefly, then release the lock before performing PostgreSQL operations - Updated removeOutdatedTables() to schedule cleanup early and avoid holding the mutex during PostgreSQL connection attempts - Removed private method declarations from DatabasePostgreSQL.h This ensures that metadata operations (like reading system.databases) are no longer blocked by failing PostgreSQL connections, improving system resilience and allowing problematic databases to be dropped even when connections fail. Co-authored-by: Kevin Michel --- .../PostgreSQL/DatabasePostgreSQL.cpp | 314 +++++++++++++----- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 8 - 2 files changed, 222 insertions(+), 100 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index e2ffc46fcf6e..3376d3e63c96 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -89,32 +89,53 @@ DatabasePostgreSQL::DatabasePostgreSQL( } -String DatabasePostgreSQL::getTableNameForLogs(const String & table_name) const +static String getTableNameForLogs(const String & database, const String & schema, const String & table_name) { - if (configuration.schema.empty()) - return fmt::format("{}.{}", configuration.database, table_name); - return fmt::format("{}.{}.{}", configuration.database, configuration.schema, table_name); + if (schema.empty()) + return fmt::format("{}.{}", database, table_name); + return fmt::format("{}.{}.{}", database, schema, table_name); } -String DatabasePostgreSQL::formatTableName(const String & table_name, bool quoted) const +String formatTableName(const String & schema, const String & table_name, bool quoted = true) { - if (configuration.schema.empty()) + if (schema.empty()) return quoted ? doubleQuoteString(table_name) : table_name; - return quoted ? fmt::format("{}.{}", doubleQuoteString(configuration.schema), doubleQuoteString(table_name)) - : fmt::format("{}.{}", configuration.schema, table_name); + return quoted ? fmt::format("{}.{}", doubleQuoteString(schema), doubleQuoteString(table_name)) + : fmt::format("{}.{}", schema, table_name); +} + + +StoragePtr fetchTable( + postgres::PoolWithFailoverPtr pool, pqxx::connection & connection, + const String & database_name, const String & schema, const String & table_name, + const String & on_conflict, ContextPtr context) +{ + auto columns_info = fetchPostgreSQLTableStructure(connection, table_name, schema).physical_columns; + if (!columns_info) + return StoragePtr{}; + + return std::make_shared( + StorageID(database_name, table_name), pool, table_name, + ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, context, schema, on_conflict); } bool DatabasePostgreSQL::empty() const { - std::lock_guard lock(mutex); + String schema; + std::unordered_set local_detached_or_dropped; + { + std::lock_guard lock(mutex); + schema = configuration.schema; + local_detached_or_dropped = detached_or_dropped; + } auto connection_holder = pool->get(); - auto tables_list = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); + auto tables_list = fetchPostgreSQLTablesList(connection_holder->get(), schema); for (const auto & table_name : tables_list) - if (!detached_or_dropped.contains(table_name)) + if (!local_detached_or_dropped.contains(table_name)) return false; return true; @@ -123,30 +144,55 @@ bool DatabasePostgreSQL::empty() const DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & /* filter_by_table_name */, bool /* skip_not_loaded */) const { - std::lock_guard lock(mutex); + String schema; + String local_database_name; + String on_conflict; + std::unordered_set local_detached_or_dropped; + bool local_cache_table; Tables tables; + { + std::lock_guard lock{mutex}; + schema = configuration.schema; + local_database_name = database_name; + on_conflict = configuration.on_conflict; + local_detached_or_dropped = detached_or_dropped; + local_cache_table = cache_tables; + if (cache_tables) + tables = cached_tables; + } /// Do not allow to throw here, because this might be, for example, a query to system.tables. /// It must not fail on case of some postgres error. try { auto connection_holder = pool->get(); - auto table_names = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); + auto & connection = connection_holder->get(); + auto table_names = fetchPostgreSQLTablesList(connection, schema); + /// First remove outdated or dropped entries + for (auto it = tables.cbegin();it != tables.cend();) + { + if (!table_names.contains(it->first) || !local_detached_or_dropped.contains(it->first)) + it = tables.erase(it); + else + ++it; + } + + /// Then fetch all non-dropped tables we still need to add for (const auto & table_name : table_names) - if (!detached_or_dropped.contains(table_name)) - tables[table_name] = fetchTable(table_name, local_context, true); + if (!local_detached_or_dropped.contains(table_name) && (!local_cache_table || !tables.contains(table_name))) + tables[table_name] = fetchTable(pool, connection, local_database_name, schema, table_name, on_conflict, local_context); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } - return std::make_unique(tables, database_name); + return std::make_unique(tables, local_database_name); } -bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const +bool checkPostgresTable(pqxx::connection & connection, const String & schema, const String & table_name) { if (table_name.contains('\'') || table_name.contains('\\')) { @@ -154,8 +200,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const "PostgreSQL table name cannot contain single quote or backslash characters, passed {}", table_name); } - auto connection_holder = pool->get(); - pqxx::nontransaction tx(connection_holder->get()); + pqxx::nontransaction tx(connection); try { @@ -165,9 +210,9 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const "FROM pg_catalog.pg_tables " "WHERE schemaname != 'pg_catalog' AND {} " "AND tablename = '{}'", - formatTableName(table_name), - (configuration.schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(configuration.schema)), - formatTableName(table_name))); + formatTableName(schema, table_name), + (schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(schema)), + formatTableName(schema, table_name))); } catch (pqxx::undefined_table const &) { @@ -185,79 +230,91 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const bool DatabasePostgreSQL::isTableExist(const String & table_name, ContextPtr /* context */) const { - std::lock_guard lock(mutex); - - if (detached_or_dropped.contains(table_name)) - return false; - - return checkPostgresTable(table_name); + String schema; + { + std::lock_guard lock(mutex); + schema = configuration.schema; + if (detached_or_dropped.contains(table_name)) + return false; + } + auto connection_holder = pool->get(); + return checkPostgresTable(connection_holder->get(), schema, table_name); } StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr local_context) const { - std::lock_guard lock(mutex); - - if (!detached_or_dropped.contains(table_name)) - return fetchTable(table_name, local_context, false); - - return StoragePtr{}; -} - - -StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr context_, bool table_checked) const -{ - if (!cache_tables || !cached_tables.contains(table_name)) + String schema; + String local_database_name; + String on_conflict; + bool table_detached_or_dropped; + StoragePtr storage{}; { - if (!table_checked && !checkPostgresTable(table_name)) - return StoragePtr{}; + std::lock_guard lock{mutex}; + schema = configuration.schema; + local_database_name = database_name; + on_conflict = configuration.on_conflict; + table_detached_or_dropped = detached_or_dropped.contains(table_name); + if (!table_detached_or_dropped && cache_tables && cached_tables.contains(table_name)) + storage = cached_tables[table_name]; + } + if (!table_detached_or_dropped) + { auto connection_holder = pool->get(); - auto columns_info = fetchPostgreSQLTableStructure(connection_holder->get(), table_name, configuration.schema).physical_columns; - - if (!columns_info) - return StoragePtr{}; - - auto storage = std::make_shared( - StorageID(database_name, table_name), pool, table_name, - ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, - context_, configuration.schema, configuration.on_conflict); - - if (cache_tables) + auto & connection = connection_holder->get(); + if (checkPostgresTable(connection, schema, table_name)) { - LOG_TEST(log, "Cached table `{}`", table_name); - cached_tables[table_name] = storage; + if (!storage) + storage = fetchTable(pool, connection, local_database_name, schema, table_name, on_conflict, local_context); + } + else + { + storage = StoragePtr{}; } - - return storage; - } - - if (table_checked || checkPostgresTable(table_name)) - { - return cached_tables[table_name]; } - /// Table does not exist anymore - cached_tables.erase(table_name); - return StoragePtr{}; + std::lock_guard lock{mutex}; + if (storage && cache_tables && !cached_tables.contains(table_name)) + cached_tables[table_name] = storage; + if (!storage && cache_tables) + cached_tables.erase(table_name); + return storage; } void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) { - auto db_disk = getDisk(); - std::lock_guard lock{mutex}; - - if (!checkPostgresTable(table_name)) - throw Exception(ErrorCodes::UNKNOWN_TABLE, - "Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL (database: {})", - getTableNameForLogs(table_name), database_name); - - if (!detached_or_dropped.contains(table_name)) - throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, - "Cannot attach PostgreSQL table {} because it already exists (database: {})", - getTableNameForLogs(table_name), database_name); + String database; + String schema; + String local_database_name; + std::unordered_set local_detached_or_dropped; + { + std::lock_guard lock{mutex}; + database = configuration.database; + schema = configuration.schema; + local_database_name = database_name; + local_detached_or_dropped = detached_or_dropped; + } + { + auto connection_holder = pool->get(); + auto & connection = connection_holder->get(); + if (!checkPostgresTable(connection, schema, table_name)) + throw Exception( + ErrorCodes::UNKNOWN_TABLE, + "Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL (database: {})", + getTableNameForLogs(database, schema, table_name), + local_database_name); + + if (!local_detached_or_dropped.contains(table_name)) + throw Exception( + ErrorCodes::TABLE_ALREADY_EXISTS, + "Cannot attach PostgreSQL table {} because it already exists (database: {})", + getTableNameForLogs(database, schema, table_name), + local_database_name); + } + std::lock_guard lock{mutex}; if (cache_tables) cached_tables[table_name] = storage; @@ -266,6 +323,7 @@ void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & t if (!persistent) return; + auto db_disk = getDisk(); fs::path table_marked_as_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); db_disk->removeFileIfExists(table_marked_as_removed); } @@ -273,13 +331,27 @@ void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & t StoragePtr DatabasePostgreSQL::detachTable(ContextPtr /* context_ */, const String & table_name) { + String database; + String schema; + { + std::lock_guard lock{mutex}; + database = configuration.database; + schema = configuration.schema; + } + { + auto connection_holder = pool->get(); + auto & connection = connection_holder->get(); + if (!checkPostgresTable(connection, schema, table_name)) + throw Exception( + ErrorCodes::UNKNOWN_TABLE, + "Cannot detach table {}, because it does not exist", + getTableNameForLogs(database, schema, table_name)); + } + std::lock_guard lock{mutex}; if (detached_or_dropped.contains(table_name)) - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot detach table {}. It is already dropped/detached", getTableNameForLogs(table_name)); - - if (!checkPostgresTable(table_name)) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot detach table {}, because it does not exist", getTableNameForLogs(table_name)); + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot detach table {}. It is already dropped/detached", getTableNameForLogs(database, schema, table_name)); if (cache_tables) cached_tables.erase(table_name); @@ -307,15 +379,29 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / if (!persistent) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP TABLE is not supported for non-persistent MySQL database"); - auto db_disk = getDisk(); - std::lock_guard lock{mutex}; + String database; + String schema; + { + std::lock_guard lock{mutex}; + database = configuration.database; + schema = configuration.schema; + } + { + auto connection_holder = pool->get(); + auto & connection = connection_holder->get(); + if (!checkPostgresTable(connection, schema, table_name)) + throw Exception( + ErrorCodes::UNKNOWN_TABLE, + "Cannot drop table {} because it does not exist", + getTableNameForLogs(database, schema, table_name)); + } - if (!checkPostgresTable(table_name)) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot drop table {} because it does not exist", getTableNameForLogs(table_name)); + std::lock_guard lock{mutex}; if (detached_or_dropped.contains(table_name)) - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is already dropped/detached", getTableNameForLogs(table_name)); + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is already dropped/detached", getTableNameForLogs(database, schema, table_name)); + auto db_disk = getDisk(); fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); db_disk->createFile(mark_table_removed); @@ -364,13 +450,24 @@ void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, Load void DatabasePostgreSQL::removeOutdatedTables() { + String schema; + { + std::lock_guard lock{mutex}; + // We schedule that immediately to make sure we schedule it even if anything fails + cleaner_task->scheduleAfter(cleaner_reschedule_ms); + // Early exit if we know we don't need to fetch the tables list at all + // this avoids holding the mutex for too long if the remote PostgreSQL server is dead. + if ((!cache_tables || cached_tables.empty()) && detached_or_dropped.empty()) + return; + schema = configuration.schema; + } std::lock_guard lock{mutex}; std::set actual_tables; try { auto connection_holder = pool->get(); - actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); + actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), schema); } catch (...) { @@ -382,7 +479,7 @@ void DatabasePostgreSQL::removeOutdatedTables() * connection period is exclusive and timeout is at least 2 seconds for * PostgreSQL. */ - cleaner_task->scheduleAfter(reschedule_error_multiplier * cleaner_reschedule_ms); + cleaner_task->scheduleAfter(reschedule_error_multiplier * cleaner_reschedule_ms, /* overwrite= */ true); return; } @@ -415,8 +512,6 @@ void DatabasePostgreSQL::removeOutdatedTables() else ++iter; } - - cleaner_task->scheduleAfter(cleaner_reschedule_ms); } @@ -445,15 +540,50 @@ ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const { + String database; + String schema; + String local_database_name; + String on_conflict; + bool table_detached_or_dropped; StoragePtr storage; { std::lock_guard lock{mutex}; - storage = fetchTable(table_name, local_context, false); + database = configuration.database; + schema = configuration.schema; + local_database_name = database_name; + on_conflict = configuration.on_conflict; + table_detached_or_dropped = detached_or_dropped.contains(table_name); + if (!table_detached_or_dropped && cache_tables && cached_tables.contains(table_name)) + storage = cached_tables[table_name]; + } + + if (!table_detached_or_dropped) + { + auto connection_holder = pool->get(); + auto & connection = connection_holder->get(); + if (checkPostgresTable(connection, schema, table_name)) + { + if (!storage) + storage = fetchTable(pool, connection, local_database_name, schema, table_name, on_conflict, local_context); + } + else + { + storage = StoragePtr {}; + } + } + + { + std::lock_guard lock{mutex}; + if (storage && cache_tables && !cached_tables.contains(table_name)) + cached_tables[table_name] = storage; + else if (!storage && cache_tables) + cached_tables.erase(table_name); } + if (!storage) { if (throw_on_error) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", getTableNameForLogs(table_name)); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", getTableNameForLogs(database, schema, table_name)); return nullptr; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 3b0d3c68226a..7cb83b117066 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -87,14 +87,6 @@ class DatabasePostgreSQL final : public IDatabase, WithContext bool persistent = true; const UUID db_uuid; - String getTableNameForLogs(const String & table_name) const; - - String formatTableName(const String & table_name, bool quoted = true) const; - - bool checkPostgresTable(const String & table_name) const; - - StoragePtr fetchTable(const String & table_name, ContextPtr context, bool table_checked) const TSA_REQUIRES(mutex); - void removeOutdatedTables(); ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; From 9830bb5f8370f01a9e3b43d1e3eefa5a275539c3 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Fri, 2 Jan 2026 11:47:58 +0100 Subject: [PATCH 36/49] Add support for integration metadata to named collections validation We need to keep track of existing named collections. In order to do that, we wish to add some metadata to each collection. The metadata keys are added as optional collection parameters for the storages and functions that validate the keys. This change allows `integration_id` and `integration_hash` keys to be present in named collections without triggering validation errors. These metadata keys are whitelisted in the validation function, allowing integrations to track which collections they own or manage without breaking existing validation logic. Changes: - Added whitelist check for `integration_id` and `integration_hash` keys in validateNamedCollection() function - These keys are now silently ignored during validation, allowing them to be stored in named collections without being listed as required or optional keys Co-authored-by: Aris Tritas --- src/Storages/NamedCollectionsHelpers.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index f63893e5fb2c..6a308c58b9c9 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -128,6 +128,11 @@ void validateNamedCollection( continue; } + if (key == "integration_id" || key == "integration_hash") + { + continue; + } + if (required_keys.contains(key)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Duplicate key {} in named collection", key); From 44fbad547bb41be0f0efb67f28b8f569341f46bb Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Fri, 2 Jan 2026 15:46:14 +0100 Subject: [PATCH 37/49] Multiple changes in PostgreSQL dictionary * Allow using DDL created named collection for PostgreSQL dictionary. Currently this fails upstream. * Ensure that TLS works. * Enforce using named collection. This patch simplifies the PostgreSQL dictionary source implementation by removing support for config file-based configuration and enforcing the use of named collections. This improves consistency, security, and enables proper TLS/SSL support for dictionary sources. Changes: - Removed validateConfigKeys() function and all config file parsing logic - Enforced named collection requirement (throws UNSUPPORTED_METHOD if not provided) - Simplified dictionary registration to use StoragePostgreSQL::processNamedCollectionResult() - Added overloaded processNamedCollectionResult() method that accepts additional_allowed_args - Changed pool creation from replicas_by_priority to single common_configuration - Removed replica support from config files (only works with named collections now) Breaking change: Users using config file-based PostgreSQL dictionary configuration must migrate to named collections. The old configuration method is no longer supported. Co-authored-by: Joe Lynch --- .../PostgreSQLDictionarySource.cpp | 137 +++--------------- src/Storages/StoragePostgreSQL.cpp | 15 +- src/Storages/StoragePostgreSQL.h | 5 + 3 files changed, 40 insertions(+), 117 deletions(-) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index e48f858e22fd..d0b2a0797782 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -37,6 +37,7 @@ namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; extern const int BAD_ARGUMENTS; + extern const int UNSUPPORTED_METHOD; } static const ValidateKeysMultiset dictionary_allowed_keys = { @@ -188,19 +189,6 @@ std::string PostgreSQLDictionarySource::toString() const return "PostgreSQL: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where); } -static void validateConfigKeys( - const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix) -{ - Poco::Util::AbstractConfiguration::Keys config_keys; - dict_config.keys(config_prefix, config_keys); - for (const auto & config_key : config_keys) - { - if (dictionary_allowed_keys.contains(config_key) || startsWith(config_key, "replica")) - continue; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); - } -} - #endif void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) @@ -219,111 +207,32 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const auto & settings = context->getSettingsRef(); std::optional dictionary_configuration; - postgres::PoolWithFailover::ReplicasConfigurationByPriority replicas_by_priority; - - bool bg_reconnect = false; auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, settings_config_prefix, context) : nullptr; - if (named_collection) - { - validateNamedCollection>(*named_collection, {}, dictionary_allowed_keys); - - StoragePostgreSQL::Configuration common_configuration; - common_configuration.host = named_collection->getOrDefault("host", ""); - common_configuration.port = named_collection->getOrDefault("port", 0); - common_configuration.username = named_collection->getOrDefault("user", ""); - common_configuration.password = named_collection->getOrDefault("password", ""); - common_configuration.database = named_collection->getAnyOrDefault({"database", "db"}, ""); - common_configuration.schema = named_collection->getOrDefault("schema", ""); - common_configuration.table = named_collection->getOrDefault("table", ""); - - dictionary_configuration.emplace(PostgreSQLDictionarySource::Configuration{ - .db = common_configuration.database, - .schema = common_configuration.schema, - .table = common_configuration.table, - .query = named_collection->getOrDefault("query", ""), - .where = named_collection->getOrDefault("where", ""), - .invalidate_query = named_collection->getOrDefault("invalidate_query", ""), - .update_field = named_collection->getOrDefault("update_field", ""), - .update_lag = named_collection->getOrDefault("update_lag", 1), - }); - - bg_reconnect = named_collection->getOrDefault("background_reconnect", false); - - replicas_by_priority[0].emplace_back(common_configuration); - } - else - { - validateConfigKeys(config, settings_config_prefix); - - StoragePostgreSQL::Configuration common_configuration; - common_configuration.host = config.getString(settings_config_prefix + ".host", ""); - common_configuration.port = config.getUInt(settings_config_prefix + ".port", 0); - common_configuration.username = config.getString(settings_config_prefix + ".user", ""); - common_configuration.password = config.getString(settings_config_prefix + ".password", ""); - common_configuration.database = config.getString(fmt::format("{}.database", settings_config_prefix), config.getString(fmt::format("{}.db", settings_config_prefix), "")); - common_configuration.schema = config.getString(fmt::format("{}.schema", settings_config_prefix), ""); - common_configuration.table = config.getString(fmt::format("{}.table", settings_config_prefix), ""); - - dictionary_configuration.emplace(PostgreSQLDictionarySource::Configuration - { - .db = common_configuration.database, - .schema = common_configuration.schema, - .table = common_configuration.table, - .query = config.getString(fmt::format("{}.query", settings_config_prefix), ""), - .where = config.getString(fmt::format("{}.where", settings_config_prefix), ""), - .invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""), - .update_field = config.getString(fmt::format("{}.update_field", settings_config_prefix), ""), - .update_lag = config.getUInt64(fmt::format("{}.update_lag", settings_config_prefix), 1) - }); - - bg_reconnect = config.getBool(fmt::format("{}.background_reconnect", settings_config_prefix), false); - - if (config.has(settings_config_prefix + ".replica")) - { - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(settings_config_prefix, config_keys); - - for (const auto & config_key : config_keys) - { - if (config_key.starts_with("replica")) - { - String replica_name = settings_config_prefix + "." + config_key; - StoragePostgreSQL::Configuration replica_configuration{common_configuration}; - - size_t priority = config.getInt(replica_name + ".priority", 0); - replica_configuration.host = config.getString(replica_name + ".host", common_configuration.host); - replica_configuration.port = config.getUInt(replica_name + ".port", common_configuration.port); - replica_configuration.username = config.getString(replica_name + ".user", common_configuration.username); - replica_configuration.password = config.getString(replica_name + ".password", common_configuration.password); - - if (replica_configuration.host.empty() || replica_configuration.port == 0 - || replica_configuration.username.empty() || replica_configuration.password.empty()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Named collection of connection parameters is missing some " - "of the parameters and no other dictionary parameters are added"); - } - - replicas_by_priority[priority].emplace_back(replica_configuration); - } - } - } - else - { - replicas_by_priority[0].emplace_back(common_configuration); - } - } - if (created_from_ddl) - { - for (const auto & [_, replicas] : replicas_by_priority) - for (const auto & replica : replicas) - context->getRemoteHostFilter().checkHostAndPort(replica.host, toString(replica.port)); - } + if (!named_collection) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "PostgreSQL dictionary source configuration must use a named collection"); + + StoragePostgreSQL::Configuration common_configuration = StoragePostgreSQL::processNamedCollectionResult( + *named_collection, context, dictionary_allowed_keys, /*require_table*/true); + + dictionary_configuration.emplace(PostgreSQLDictionarySource::Configuration{ + .db = common_configuration.database, + .schema = common_configuration.schema, + .table = common_configuration.table, + .query = named_collection->getOrDefault("query", ""), + .where = named_collection->getOrDefault("where", ""), + .invalidate_query = named_collection->getOrDefault("invalidate_query", ""), + .update_field = named_collection->getOrDefault("update_field", ""), + .update_lag = named_collection->getOrDefault("update_lag", 1), + }); + + for (const auto & [host, port] : common_configuration.addresses) + context->getRemoteHostFilter().checkHostAndPort(host, toString(port)); auto pool = std::make_shared( - replicas_by_priority, + common_configuration, settings[Setting::postgresql_connection_pool_size], settings[Setting::postgresql_connection_pool_wait_timeout], settings[Setting::postgresql_connection_pool_retries], @@ -331,7 +240,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) settings[Setting::postgresql_connection_attempt_timeout], static_cast(settings[Setting::postgresql_connection_pool_ssl_mode]), static_cast(settings[Setting::postgresql_connection_pool_ssl_root_cert]), - bg_reconnect); + named_collection->getOrDefault("background_reconnect", false)); return std::make_unique(dict_struct, dictionary_configuration.value(), pool, std::make_shared(sample_block)); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index eb55ccbafc82..bed95eaabed5 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -531,15 +531,24 @@ SinkToStoragePtr StoragePostgreSQL::write( return std::make_shared(metadata_snapshot, pool->get(), remote_table_name, remote_table_schema, on_conflict); } -StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult(const NamedCollection & named_collection, ContextPtr context_, bool require_table) +StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult(const NamedCollection & named_collection, ContextPtr context_, bool require_table) { + return processNamedCollectionResult(named_collection, context_, {}, require_table); + +} + + +StoragePostgreSQL::Configuration StoragePostgreSQL::processNamedCollectionResult(const NamedCollection & named_collection, ContextPtr context_, const ValidateKeysMultiset & additional_allowed_args, bool require_table) { StoragePostgreSQL::Configuration configuration; ValidateKeysMultiset required_arguments = {"user", "username", "password", "database", "db"}; if (require_table) required_arguments.insert("table"); - validateNamedCollection>( - named_collection, required_arguments, {"schema", "on_conflict", "addresses_expr", "host", "hostname", "port", "use_table_cache", "ssl_root_cert", "ssl_mode"}); + ValidateKeysMultiset optional_args = {"schema", "on_conflict", "addresses_expr", "host", "hostname", "port", "use_table_cache", "ssl_root_cert", "ssl_mode"}; + for (const auto & arg : additional_allowed_args) + optional_args.insert(arg); + + validateNamedCollection>(named_collection, required_arguments, optional_args); configuration.addresses_expr = named_collection.getOrDefault("addresses_expr", ""); if (configuration.addresses_expr.empty()) diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index b67aeb9e52a1..c7f9c54d05c7 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace Poco { @@ -71,6 +72,10 @@ class StoragePostgreSQL final : public IStorage static Configuration processNamedCollectionResult(const NamedCollection & named_collection, ContextPtr context_, bool require_table = true); + static Configuration processNamedCollectionResult(const NamedCollection &named_collection, ContextPtr context_, + const ValidateKeysMultiset & + additional_allowed_args, bool require_table = true); + static ColumnsDescription getTableStructureFromData( const postgres::PoolWithFailoverPtr & pool_, const String & table, From 30eaccb119e522955b30f5f370f9e8ec6550e971 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 10:50:31 +0100 Subject: [PATCH 38/49] Ignore unreadable sensors This is a convenience patch to avoid polluting logs on developer laptops with GPU drivers that advertise non-working thermal sensors. The patch disables initialization of EDAC (Error Detection And Correction) and hardware monitoring chip sensors by removing the calls to openEDAC() and openSensorsChips() from the AsynchronousMetrics constructor. Co-authored-by: Kevin Michel --- src/Common/AsynchronousMetrics.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index bb5f62c162a6..5c34e71213e4 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -171,8 +171,6 @@ AsynchronousMetrics::AsynchronousMetrics( openSensors(); openBlockDevices(); - openEDAC(); - openSensorsChips(); #endif } From a9ff91b2d971e9f157705453b389225e58874afe Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 12:04:39 +0100 Subject: [PATCH 39/49] Fix tcp_port_secure from ZK This patch fixes getTCPPortSecure() to read the port from the registered server ports map instead of directly from the configuration file. This ensures it works correctly when the port is obtained from ZooKeeper or other dynamic sources, not just from static configuration. The patch also improves thread safety by adding proper mutex protection for the server_ports map and introduces a new tryGetServerPort() helper method for non-throwing port lookups. Co-authored-by: Kevin Michel --- src/Interpreters/Context.cpp | 23 +++++++++++++++-------- src/Interpreters/Context.h | 2 ++ 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4ec44f0b66bd..f004de482430 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -386,6 +386,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; extern const int CLUSTER_DOESNT_EXIST; + extern const int BAD_GET; extern const int SET_NON_GRANTED_ROLE; extern const int UNKNOWN_DISK; extern const int UNKNOWN_READ_METHOD; @@ -4767,25 +4768,31 @@ UInt16 Context::getTCPPort() const return config.getInt("tcp_port", DBMS_DEFAULT_PORT); } +std::optional Context::tryGetServerPort(const String & port_name) const +{ + SharedLockGuard lock(shared->mutex); + auto it = shared->server_ports.find(port_name); + if (it == shared->server_ports.end()) + return {}; + return it->second; +} + std::optional Context::getTCPPortSecure() const { - const auto & config = getConfigRef(); - if (config.has("tcp_port_secure")) - return config.getInt("tcp_port_secure"); - return {}; + return tryGetServerPort("tcp_port_secure"); } void Context::registerServerPort(String port_name, UInt16 port) { + std::lock_guard lock(shared->mutex); shared->server_ports.emplace(std::move(port_name), port); } UInt16 Context::getServerPort(const String & port_name) const { - auto it = shared->server_ports.find(port_name); - if (it == shared->server_ports.end()) - throw Exception(ErrorCodes::CLUSTER_DOESNT_EXIST, "There is no port named {}", port_name); - return it->second; + if (auto port = tryGetServerPort(port_name)) + return *port; + throw Exception(ErrorCodes::BAD_GET, "There is no port named {}", port_name); } size_t Context::getMaxPendingMutationsToWarn() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3d573d62173f..f11fdbfd53b9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1108,6 +1108,8 @@ class Context: public ContextData, public std::enable_shared_from_this UInt16 getServerPort(const String & port_name) const; + std::optional tryGetServerPort(const String & port_name) const; + /// For methods below you may need to acquire the context lock by yourself. ContextMutablePtr getQueryContext() const; From 8cdfb644102bf2ecae905342d93bf600505185af Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 13:01:33 +0100 Subject: [PATCH 40/49] Disable thread fuzzer This patch disables the ThreadFuzzer pthread wrapping feature by always setting THREAD_FUZZER_WRAP_PTHREAD to 0, regardless of platform or sanitizer settings. The pthread wrapping feature has compatibility issues with newer glibc versions (especially glibc 2.36+) and is a testing feature that should not be enabled in production builds. This patch simplifies the code by removing conditional compilation logic and ensuring consistent behavior across all platforms. Co-authored-by: Kevin Michel --- src/Common/ThreadFuzzer.cpp | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Common/ThreadFuzzer.cpp b/src/Common/ThreadFuzzer.cpp index 5e2aa35d2b6c..c5df91b4a4e1 100644 --- a/src/Common/ThreadFuzzer.cpp +++ b/src/Common/ThreadFuzzer.cpp @@ -22,13 +22,7 @@ #include "config.h" // USE_JEMALLOC - -/// We will also wrap some thread synchronization functions to inject sleep/migration before or after. -#if defined(OS_LINUX) && !defined(THREAD_SANITIZER) && !defined(MEMORY_SANITIZER) - #define THREAD_FUZZER_WRAP_PTHREAD 1 -#else - #define THREAD_FUZZER_WRAP_PTHREAD 0 -#endif +#define THREAD_FUZZER_WRAP_PTHREAD 0 #if THREAD_FUZZER_WRAP_PTHREAD # define FOR_EACH_WRAPPED_FUNCTION(M) \ From b950a5e2b93579b2980265abc2f31ba90561046e Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 14:06:22 +0100 Subject: [PATCH 41/49] Disable replicas_status endpoint This patch removes the default registration of the /replicas_status HTTP endpoint to reduce the default attack surface and prevent exposure of replication state information by default. The /replicas_status endpoint provides information about the status of replicated MergeTree tables, including replication lag and detailed state information. While useful for monitoring, this information should not be exposed by default for security and privacy reasons. Co-authored-by: Kevin Michel --- src/Server/HTTPHandlerFactory.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 98b43f0655a5..61274105069e 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -314,12 +314,6 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS factory.addPathToHints("/ping"); factory.addHandler(ping_handler); - auto replicas_status_handler = std::make_shared>(server); - replicas_status_handler->attachNonStrictPath("/replicas_status"); - replicas_status_handler->allowGetAndHeadRequest(); - factory.addPathToHints("/replicas_status"); - factory.addHandler(replicas_status_handler); - auto play_handler = std::make_shared>(server); play_handler->attachNonStrictPath("/play"); play_handler->allowGetAndHeadRequest(); From 6c950c20d1d854df0bc99276b77f338f61b525e2 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 15:33:31 +0100 Subject: [PATCH 42/49] Fix swap drift Even with low swappiness, sometimes ClickHouse pages go to swap. By itself, it's not a problem, unused pages can move to swap without any issue. However, this interacts badly with the memory tracker of ClickHouse. The memory tracker is in charge of limiting the total memory usage of the server without interrupting the service (unlike the OOM killer or an earlyOOM daemon). Instead, individual queries are delayed or rejected until more memory is available. The tracker normally only looks at the RSS of the process, which excludes the swap. If part of ClickHouse memory is moved to swap, then the tracker stops counting that memory towards the limit and allows more memory allocations. Then a little bit more memory is moved to swap, and this cycle keeps going until all swap is consumed. In case of low swappiness configuration of the system it can take days or weeks before ClickHouse consumes all swap. At that point, ClickHouse is still below its configured maximum ram usage, as configured. But, if some swapped page suddenly needs to move back to ram, then we have a real problem: there is not enough room in ram because all processes were allocated a portion of it and there is no room in swap because ClickHouse slowly consumed everything. We have to move things around but no room to move, it's swap hell. Fix it by taking in account the swapped memory of the process in addition to its RSS. To get this data, ClickHouse needs to be modified to look at `/proc/self/status` instead of `/proc/self/statm`, and adapt to the different file format. (statm does not contain swap information). As a bonus, the swapped memory is exposed as a metric like the other memory metrics already present. Impact: - Memory tracker now accounts for swap memory, preventing swap drift - Prevents swap exhaustion and "swap hell" scenarios - Memory limits are correctly enforced including swap usage - MemorySwap metric available for monitoring - More accurate memory tracking in systems with swap enabled Co-authored-by: Kevin Michel --- src/Common/AsynchronousMetrics.cpp | 7 ++- src/Common/MemoryStatisticsOS.cpp | 86 ++++++++++++++++++++++-------- src/Common/MemoryStatisticsOS.h | 3 +- src/Common/MemoryTracker.cpp | 28 ++++++++++ src/Common/MemoryTracker.h | 2 + src/Common/MemoryWorker.cpp | 18 +++++-- src/Core/ServerSettings.cpp | 4 +- 7 files changed, 117 insertions(+), 31 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 5c34e71213e4..561d061d52de 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -992,6 +992,8 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) "The amount of memory used by the server process, that is also shared by another processes, in bytes." " ClickHouse does not use shared memory, but some memory can be labeled by OS as shared for its own reasons." " This metric does not make a lot of sense to watch, and it exists only for completeness reasons."}; + new_values["MemorySwap"] = {data.swap, + "The amount of memory that was moved from physical ram to disk, in bytes."}; #endif new_values["MemoryCode"] = { data.code, "The amount of virtual memory mapped for the pages of machine code of the server process, in bytes." }; @@ -1000,8 +1002,9 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; - if (update_rss) - MemoryTracker::updateRSS(data.resident); + if (update_rss) { + MemoryTracker::updateRSSPlusSwap(data.resident + data.swap); + } } { diff --git a/src/Common/MemoryStatisticsOS.cpp b/src/Common/MemoryStatisticsOS.cpp index 5dddeb8310d4..a1a948f7f8d7 100644 --- a/src/Common/MemoryStatisticsOS.cpp +++ b/src/Common/MemoryStatisticsOS.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes extern const int CANNOT_CLOSE_FILE; } -static constexpr auto filename = "/proc/self/statm"; +static constexpr auto filename = "/proc/self/status"; MemoryStatisticsOS::MemoryStatisticsOS() { @@ -61,9 +61,10 @@ MemoryStatisticsOS::~MemoryStatisticsOS() MemoryStatisticsOS::Data MemoryStatisticsOS::get() const { - Data data; + Data data{}; + data.swap = 0; - constexpr size_t buf_size = 1024; + constexpr size_t buf_size = 2048; char buf[buf_size]; ssize_t res = 0; @@ -84,27 +85,68 @@ MemoryStatisticsOS::Data MemoryStatisticsOS::get() const break; } while (true); + // Parse /proc/self/status file format (key: value) ReadBufferFromMemory in(buf, res); - uint64_t unused; - readIntText(data.virt, in); - skipWhitespaceIfAny(in); - readIntText(data.resident, in); - skipWhitespaceIfAny(in); - readIntText(data.shared, in); - skipWhitespaceIfAny(in); - readIntText(data.code, in); - skipWhitespaceIfAny(in); - readIntText(unused, in); - skipWhitespaceIfAny(in); - readIntText(data.data_and_stack, in); - - size_t page_size = static_cast(::getPageSize()); - data.virt *= page_size; - data.resident *= page_size; - data.shared *= page_size; - data.code *= page_size; - data.data_and_stack *= page_size; + while (!in.eof()) + { + String name; + readStringUntilWhitespace(name, in); + skipWhitespaceIfAny(in, true); + + uint64_t value_kb = 0; + readText(value_kb, in); + + if (!value_kb) + { + skipToNextLineOrEOF(in); + continue; + } + + skipWhitespaceIfAny(in, true); + + // Check if line ends (some entries don't have kB suffix) + if (*in.position() == '\n') + { + skipToNextLineOrEOF(in); + continue; + } + + assertString("kB", in); + skipToNextLineOrEOF(in); + + uint64_t bytes = value_kb * 1024; + + if (name == "VmSize:") + { + data.virt = bytes; + } + else if (name == "VmRSS:") + { + data.resident = bytes; + } + else if (name == "RssAnon:") + { + // Approximation for shared, as status doesn't have exact shared + data.shared = bytes; + } + else if (name == "VmSwap:") + { + data.swap = bytes; + } + else if (name == "VmExe:") + { + data.code = bytes; + } + else if (name == "VmData:") + { + data.data_and_stack = bytes; + } + else if (name == "VmStk:") + { + data.data_and_stack += bytes; + } + } return data; } diff --git a/src/Common/MemoryStatisticsOS.h b/src/Common/MemoryStatisticsOS.h index 8eb6c871a991..73b9805b2e48 100644 --- a/src/Common/MemoryStatisticsOS.h +++ b/src/Common/MemoryStatisticsOS.h @@ -9,7 +9,7 @@ namespace DB { -/** Opens a file /proc/self/statm. Keeps it open and reads memory statistics via 'pread'. +/** Opens a file /proc/self/status. Keeps it open and reads memory statistics via 'pread'. * This is Linux specific. * See: man procfs * @@ -28,6 +28,7 @@ class MemoryStatisticsOS uint64_t resident; #if defined(OS_LINUX) uint64_t shared; + uint64_t swap; #endif uint64_t code; uint64_t data_and_stack; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index cacc2dd43273..98118688e290 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -611,6 +611,11 @@ void MemoryTracker::updateRSS(Int64 rss_) total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); } +void MemoryTracker::updateRSSPlusSwap(Int64 rss_plus_swap_) +{ + total_memory_tracker.rss.store(rss_plus_swap_, std::memory_order_relaxed); +} + void MemoryTracker::updateAllocated(Int64 allocated_, bool log_change) { Int64 new_amount = allocated_; @@ -634,6 +639,29 @@ void MemoryTracker::updateAllocated(Int64 allocated_, bool log_change) total_memory_tracker.updatePeak(new_amount, log_memory_usage); } +void MemoryTracker::updateAllocatedPlusSwap(Int64 allocated_plus_swap_, bool log_change) +{ + Int64 new_amount = allocated_plus_swap_; + if (log_change) + LOG_INFO( + getLogger("MemoryTracker"), + "Correcting the value of global memory tracker from {} to {}", + ReadableSize(total_memory_tracker.amount.load(std::memory_order_relaxed)), + ReadableSize(allocated_plus_swap_)); + + auto current_amount = total_memory_tracker.amount.exchange(new_amount, std::memory_order_relaxed); + total_memory_tracker.uncorrected_amount += (current_amount - total_memory_tracker.last_corrected_amount); + total_memory_tracker.last_corrected_amount = new_amount; + CurrentMetrics::set(CurrentMetrics::MemoryTrackingUncorrected, total_memory_tracker.uncorrected_amount); + + auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); + if (metric_loaded != CurrentMetrics::end()) + CurrentMetrics::set(metric_loaded, new_amount); + + bool log_memory_usage = true; + total_memory_tracker.updatePeak(new_amount, log_memory_usage); +} + void MemoryTracker::setSoftLimit(Int64 value) { soft_limit.store(value, std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 7384f9157c97..c426c3d43463 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -282,7 +282,9 @@ class MemoryTracker /// update values based on external information (e.g. jemalloc's stat) static void updateRSS(Int64 rss_); + static void updateRSSPlusSwap(Int64 rss_plus_swap_); static void updateAllocated(Int64 allocated_, bool log_change); + static void updateAllocatedPlusSwap(Int64 allocated_plus_swap_, bool log_change); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index f19508b6a331..54083fe5de2a 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -321,6 +322,11 @@ void MemoryWorker::backgroundThread() std::chrono::milliseconds chrono_period_ms{period_ms}; [[maybe_unused]] bool first_run = true; std::unique_lock lock(mutex); + +#if defined(OS_LINUX) + MemoryStatisticsOS memory_stat; +#endif + while (true) { cv.wait_for(lock, chrono_period_ms, [this] { return shutdown; }); @@ -328,9 +334,13 @@ void MemoryWorker::backgroundThread() return; Stopwatch total_watch; + size_t swap_bytes = 0; +#if defined(OS_LINUX) + swap_bytes = memory_stat.get().swap; +#endif Int64 resident = getMemoryUsage(); - MemoryTracker::updateRSS(resident); + MemoryTracker::updateRSSPlusSwap(resident + swap_bytes); if (page_cache) page_cache->autoResize(std::max(resident, total_memory_tracker.get()), total_memory_tracker.getHardLimit()); @@ -349,9 +359,9 @@ void MemoryWorker::backgroundThread() /// - MemoryTracker stores a negative value /// - `correct_tracker` is set to true if (unlikely(first_run || total_memory_tracker.get() < 0)) - MemoryTracker::updateAllocated(resident, /*log_change=*/true); + MemoryTracker::updateAllocatedPlusSwap(resident + swap_bytes, /*log_change=*/true); else if (correct_tracker) - MemoryTracker::updateAllocated(resident, /*log_change=*/false); + MemoryTracker::updateAllocatedPlusSwap(resident + swap_bytes, /*log_change=*/false); #else /// we don't update in the first run if we don't have jemalloc /// because we can only use resident memory information @@ -359,7 +369,7 @@ void MemoryWorker::backgroundThread() /// so we rather ignore the potential difference caused by allocated memory /// before MemoryTracker initialization if (unlikely(total_memory_tracker.get() < 0) || correct_tracker) - MemoryTracker::updateAllocated(resident, /*log_change=*/false); + MemoryTracker::updateAllocatedPlusSwap(resident + swap_bytes, /*log_change=*/false); #endif ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index b1cf1c59e5e2..2a7294aa60ec 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1027,8 +1027,8 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ DECLARE(UInt64, memory_worker_period_ms, 0, R"( Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source )", 0) \ - DECLARE(Bool, memory_worker_correct_memory_tracker, 0, R"( - Whether background memory worker should correct internal memory tracker based on the information from external sources like jemalloc and cgroups +DECLARE(Bool, memory_worker_correct_memory_tracker, 0, R"( +Whether background memory worker should correct internal memory tracker based on the information from external sources like jemalloc, cgroups and /proc/self/status, tracking swap memory as well )", 0) \ DECLARE(Bool, memory_worker_use_cgroup, true, "Use current cgroup memory usage information to correct memory tracking.", 0) \ DECLARE(Bool, disable_insertion_and_mutation, false, R"( From d9e91137ce85029494a28a018dd5638e7a67ab94 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 16:02:42 +0100 Subject: [PATCH 43/49] Fix ZK node leak after create delete table When a replicated table is created and then deleted, the immediate parent ZooKeeper znode may remain empty and not be cleaned up, causing a node leak in ZooKeeper. This can lead to accumulation of orphaned empty znodes over time, polluting the ZooKeeper namespace. The existing `dropAncestorZnodesIfNeeded()` method in `TableZnodeInfo` removes ancestor znodes from the table path up to `path_prefix_for_drop`, but it may not handle the immediate parent znode in all cases. Fix by adding a new method `dropAncestorTableZnodeIfNeeded()` that specifically removes the immediate parent znode of the table path if it becomes empty after table deletion. This complements the existing cleanup logic and ensures no orphaned znodes are left behind. Co-authored-by: Joe Lynch --- src/Storages/StorageReplicatedMergeTree.cpp | 15 +++++++++++++++ src/Storages/StorageReplicatedMergeTree.h | 3 +++ 2 files changed, 18 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7efc6a4fcc11..7af04f5188c0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1648,9 +1648,24 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper } } + dropAncestorTableZnodeIfNeeded(zookeeper, zookeeper_path, logger); + return completely_removed; } +bool StorageReplicatedMergeTree::dropAncestorTableZnodeIfNeeded(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, LoggerPtr logger) +{ + size_t i = zookeeper_path.find_last_of('/'); + const String path_to_remove = zookeeper_path.substr(0, i); + const Coordination::Error code = zookeeper->tryRemove(path_to_remove); + if (code == Coordination::Error::ZOK) { + LOG_INFO(logger, "Removed ancestor table znode {}", path_to_remove); + return true; + } + LOG_INFO(logger, "Did not remove ancestor table znode {}, code: {}", path_to_remove, code); + return false; +} + /** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/metadata). * If not, throw an exception. diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 6a76f234b8ae..e5ad246e76aa 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -262,6 +262,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData zkutil::ZooKeeperPtr zookeeper, const TableZnodeInfo & zookeeper_info2, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, LoggerPtr logger); + /// Removes parent node from ZooKeeper if it is empty + static bool dropAncestorTableZnodeIfNeeded(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, LoggerPtr logger); + /// Schedules job to execute in background pool (merge, mutate, drop range and so on) bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; From a72c739ba80950c7c032fa8c78ffa8bc090c2583 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 16:18:45 +0100 Subject: [PATCH 44/49] Remove SSBS check from `NO_ARMV81_OR_HIGHER` The CPU validation check for ARMv8.1+ was requiring both `atomic` and `ssbs` features to be present in /proc/cpuinfo. However, SSBS (Speculative Store Bypass Safe) is optional in ARMv8.0 and only mandatory in ARMv8.5, which means some valid ARMv8.1 CPUs may not have this feature. This caused false rejections during build validation, preventing builds on legitimate ARMv8.1+ CPUs that don't have SSBS. The `atomic` feature (Large System Extensions - LSE) is a sufficient indicator of ARMv8.1+ support and doesn't have the same optionality issues. Remove the SSBS requirement from the validation check, keeping only the `atomic` check. This allows ARMv8.1 CPUs without SSBS to pass validation while still ensuring the build machine can run intermediate binaries (protoc, llvm-tablegen) that require ARMv8.1+ features. --- cmake/cpu_features.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index fa58a739611c..68267494e760 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -104,7 +104,7 @@ elseif (ARCH_AARCH64) # jscvt fcma lrcpc dcpop sha3 sm3 sm4 asimddp sha512 sve asimdfhm dit uscat ilrcpc flagm ssbs paca pacg dcpodp svei8mm svebf16 i8mm # bf16 dgh rng") execute_process( - COMMAND grep -P "^(?=.*atomic)(?=.*ssbs)" /proc/cpuinfo + COMMAND grep -P "^(?=.*atomic)" /proc/cpuinfo OUTPUT_VARIABLE FLAGS) if (NOT FLAGS) MESSAGE(FATAL_ERROR "The build machine does not satisfy the minimum CPU requirements, try to run cmake with -DNO_ARMV81_OR_HIGHER=1") From 6edb6127995b950c6d37433f53f492c694253942 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Mon, 5 Jan 2026 16:36:11 +0100 Subject: [PATCH 45/49] Enable curl ipv6 Make IPv6 support in curl configurable via CMake option, enabled by default. Previously, IPv6 support was hardcoded in curl_config.h, making it impossible to disable IPv6 at build time. Add a CMake option `ENABLE_IPV6` that controls IPv6 support in curl, with a default value of 1 (enabled). This maintains the current behavior while providing the flexibility to disable IPv6 for environments that don't support it or have specific requirements. --- contrib/curl-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index c616b7d0471e..6585b511e271 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -1,4 +1,5 @@ option (ENABLE_CURL "Enable curl" ${ENABLE_LIBRARIES}) +option (ENABLE_IPV6 "Enable IPv6" 1) if (NOT ENABLE_CURL) message(STATUS "Not using curl") @@ -192,6 +193,7 @@ target_compile_definitions (_curl PRIVATE BUILDING_LIBCURL CURL_HIDDEN_SYMBOLS libcurl_EXPORTS + ENABLE_IPV6 OPENSSL_NO_ENGINE CURL_OS="${CMAKE_SYSTEM_NAME}" ) From 8faeb0f04eeb8d3cfd72010159f6534fd37bbbc4 Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 6 Jan 2026 12:33:25 +0100 Subject: [PATCH 46/49] Allow disabling of individual dictionary sources Add compile-time flags to enable/disable individual dictionary sources, allowing builds to exclude specific sources to reduce dependencies, binary size, and attack surface. This provides flexibility for creating minimal builds or excluding dictionary sources that are not needed in specific deployments. Previously, all dictionary sources were always compiled and registered, regardless of whether they were needed. This patch adds conditional compilation guards around each dictionary source registration, controlled by CMake defines that default to enabled (maintaining backward compatibility). Co-authored-by: Joe Lynch --- src/Common/config.h.in | 15 ++++++++++ .../registerCacheDictionaries.cpp | 2 +- src/Dictionaries/registerDictionaries.cpp | 28 +++++++++++++++++++ 3 files changed, 44 insertions(+), 1 deletion(-) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 4b8d53cefdbb..71874e3c4ae8 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -81,6 +81,21 @@ #cmakedefine01 USE_YTSAURUS #cmakedefine01 USE_CLIENT_AI +#cmakedefine01 REGISTER_DICTIONARY_LAYOUT_SSD +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_CASSANDRA +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_CLICKHOUSE +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_EXECUTABLE +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_EXECUTABLEPOOL +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_FILE +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_HTTP +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_JDBC +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_LIBRARY +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_MONGODB +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_MYSQL +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_POSTGRESQL +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_REDIS +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_XDBC +#cmakedefine01 REGISTER_DICTIONARY_SOURCE_YAMLREGEXPTREE /// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO. /// That's why we use absolute paths. diff --git a/src/Dictionaries/registerCacheDictionaries.cpp b/src/Dictionaries/registerCacheDictionaries.cpp index 05606cb80acc..f64a5209bbb0 100644 --- a/src/Dictionaries/registerCacheDictionaries.cpp +++ b/src/Dictionaries/registerCacheDictionaries.cpp @@ -280,7 +280,7 @@ void registerDictionaryCache(DictionaryFactory & factory) factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true); -#if defined(OS_LINUX) || defined(OS_FREEBSD) +#if REGISTER_DICTIONARY_LAYOUT_SSD && (defined(OS_LINUX) || defined(OS_FREEBSD)) auto create_simple_ssd_cache_layout = [=](const std::string & full_name, const DictionaryStructure & dict_struct, diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index a101f9aef289..f1808979e7a0 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -42,20 +42,48 @@ void registerDictionaries() { auto & source_factory = DictionarySourceFactory::instance(); registerDictionarySourceNull(source_factory); +#if REGISTER_DICTIONARY_SOURCE_FILE registerDictionarySourceFile(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_MYSQL registerDictionarySourceMysql(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_CLICKHOUSE registerDictionarySourceClickHouse(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_MONGODB registerDictionarySourceMongoDB(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_REDIS registerDictionarySourceRedis(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_CASSANDRA registerDictionarySourceCassandra(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_XDBC registerDictionarySourceXDBC(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_JDBC registerDictionarySourceJDBC(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_POSTGRESQL registerDictionarySourcePostgreSQL(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_EXECUTABLE registerDictionarySourceExecutable(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_EXECUTABLEPOOL registerDictionarySourceExecutablePool(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_HTTP registerDictionarySourceHTTP(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_LIBRARY registerDictionarySourceLibrary(source_factory); +#endif +#if REGISTER_DICTIONARY_SOURCE_YAMLREGEXPTREE registerDictionarySourceYAMLRegExpTree(source_factory); +#endif registerDictionarySourceYTsaurus(source_factory); } From b114cbb7f3de2c694593deaedc1f2f6e02ef92cf Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 6 Jan 2026 15:03:50 +0100 Subject: [PATCH 47/49] Add early fetch pool When adding a replica to an existing cluster, the replica will add many GET_PART tasks to its replication queue. These tasks are in charge of downloading the data that existed before the creation of the replica. Meanwhile, if the cluster is still receiving updates, new GET_PART tasks will be created to add the extra data. These new parts are usually small, and in a cluster where the inserts are a bit too frequent, we can have a lot of them. However, there is a limit to the number of GET_PART tasks that a replica will simultaneously execute (by default 8). The early GET_PART tasks, for existing data, are often very large and can take a lot of time to execute. If there are more existing parts than the maximum number of simultanenous downloads, then the early parts will take all slots in the pool for download tasks and the small additional GET_PART tasks will have to wait for the very large early parts to be downloaded. The accumulation of many small GET_PART tasks then becomes an issue for ZooKeeper, each of these tasks is a znode and it's quite easy to end up with >100k pending tasks for a single table. We cannot simply increase the download pool size. A large cluster will always have more existing large parts than the pool size, unless we configure the pool with an unreasonably large size. However, we can split the pool in two parts: one pool for the existing parts that are added early to sync the replica, and another pool for the normal GET_PART tasks that happen because of normal inserts. Co-authored-by: Kevin Michel --- src/Common/CurrentMetrics.cpp | 2 + src/Core/ServerSettings.cpp | 5 +++ src/Interpreters/Context.cpp | 26 +++++++++++ src/Interpreters/Context.h | 1 + .../MergeTree/BackgroundJobsAssignee.cpp | 8 ++++ .../MergeTree/BackgroundJobsAssignee.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 1 + src/Storages/MergeTree/MergeTreeSettings.cpp | 4 ++ src/Storages/StorageReplicatedMergeTree.cpp | 45 ++++++++++++++++--- 9 files changed, 86 insertions(+), 7 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index c7130ea42d55..1940021f96d6 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -19,6 +19,8 @@ M(BackgroundMergesAndMutationsPoolSize, "Limit on number of active merges and mutations in an associated background pool") \ M(BackgroundFetchesPoolTask, "Number of active fetches in an associated background pool") \ M(BackgroundFetchesPoolSize, "Limit on number of simultaneous fetches in an associated background pool") \ + M(BackgroundEarlyFetchesPoolTask, "Number of active early fetches in an associated background pool") \ + M(BackgroundEarlyFetchesPoolSize, "Limit on number of simultaneous early fetches in an associated background pool") \ M(BackgroundCommonPoolTask, "Number of active tasks in an associated background pool") \ M(BackgroundCommonPoolSize, "Limit on number of tasks in an associated background pool") \ M(BackgroundMovePoolTask, "Number of active tasks in BackgroundProcessingPool for moves") \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 2a7294aa60ec..07ca3a20fb65 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -812,6 +812,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ )", 0) \ DECLARE(UInt64, background_move_pool_size, 8, R"(The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.)", 0) \ DECLARE(UInt64, background_fetches_pool_size, 16, R"(The maximum number of threads that will be used for fetching data parts from another replica for [*MergeTree-engine](/engines/table-engines/mergetree-family) tables in the background.)", 0) \ + DECLARE(UInt64, background_early_fetches_pool_size, 8, R"(The maximum number of threads that will be used for early fetching data parts from another replica for *MergeTree-engine tables in a background.)", 0) \ DECLARE(UInt64, background_common_pool_size, 8, R"(The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for [*MergeTree-engine](/engines/table-engines/mergetree-family) tables in the background.)", 0) \ DECLARE(UInt64, background_buffer_flush_schedule_pool_size, 16, R"(The maximum number of threads that will be used for performing flush operations for [Buffer-engine tables](/engines/table-engines/special/buffer) in the background.)", 0) \ DECLARE(UInt64, background_schedule_pool_size, 512, R"(The maximum number of threads that will be used for constantly executing some lightweight periodic operations for replicated tables, Kafka streaming, and DNS cache updates.)", 0) \ @@ -1165,6 +1166,7 @@ void ServerSettingsImpl::loadSettingsFromConfig(const Poco::Util::AbstractConfig "background_merges_mutations_scheduling_policy", "background_move_pool_size", "background_fetches_pool_size", + "background_early_fetches_pool_size", "background_common_pool_size", "background_buffer_flush_schedule_pool_size", "background_schedule_pool_size", @@ -1351,6 +1353,9 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam changeable_settings.insert( {"background_fetches_pool_size", {std::to_string(context->getFetchesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); + changeable_settings.insert( + {"background_early_fetches_pool_size", + {std::to_string(context->getEarlyFetchesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); changeable_settings.insert( {"background_common_pool_size", {std::to_string(context->getCommonExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}}); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f004de482430..a87817f8a91a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -209,6 +209,8 @@ namespace CurrentMetrics extern const Metric BackgroundMergesAndMutationsPoolSize; extern const Metric BackgroundFetchesPoolTask; extern const Metric BackgroundFetchesPoolSize; + extern const Metric BackgroundEarlyFetchesPoolTask; + extern const Metric BackgroundEarlyFetchesPoolSize; extern const Metric BackgroundCommonPoolTask; extern const Metric BackgroundCommonPoolSize; extern const Metric MarksLoaderThreads; @@ -333,6 +335,7 @@ namespace ServerSetting extern const ServerSettingsUInt64 background_common_pool_size; extern const ServerSettingsUInt64 background_distributed_schedule_pool_size; extern const ServerSettingsUInt64 background_fetches_pool_size; + extern const ServerSettingsUInt64 background_early_fetches_pool_size; extern const ServerSettingsFloat background_merges_mutations_concurrency_ratio; extern const ServerSettingsString background_merges_mutations_scheduling_policy; extern const ServerSettingsUInt64 background_message_broker_schedule_pool_size; @@ -630,6 +633,7 @@ struct ContextSharedPart : boost::noncopyable MergeMutateBackgroundExecutorPtr merge_mutate_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr moves_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr fetch_executor TSA_GUARDED_BY(background_executors_mutex); + OrdinaryBackgroundExecutorPtr early_fetch_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr common_executor TSA_GUARDED_BY(background_executors_mutex); RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml @@ -868,6 +872,7 @@ struct ContextSharedPart : boost::noncopyable SHUTDOWN(log, "merges executor", merge_mutate_executor, wait()); SHUTDOWN(log, "fetches executor", fetch_executor, wait()); + SHUTDOWN(log, "early fetches executor", early_fetch_executor, wait()); SHUTDOWN(log, "moves executor", moves_executor, wait()); SHUTDOWN(log, "common executor", common_executor, wait()); @@ -6563,6 +6568,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() String background_merges_mutations_scheduling_policy = server_settings[ServerSetting::background_merges_mutations_scheduling_policy]; size_t background_move_pool_size = server_settings[ServerSetting::background_move_pool_size]; size_t background_fetches_pool_size = server_settings[ServerSetting::background_fetches_pool_size]; + size_t background_early_fetches_pool_size = server_settings[ServerSetting::background_early_fetches_pool_size]; size_t background_common_pool_size = server_settings[ServerSetting::background_common_pool_size]; /// With this executor we can execute more tasks than threads we have @@ -6610,6 +6616,20 @@ void Context::initializeBackgroundExecutorsIfNeeded() ); LOG_INFO(shared->log, "Initialized background executor for fetches with num_threads={}, num_tasks={}", background_fetches_pool_size, background_fetches_pool_size); + shared->early_fetch_executor = std::make_shared + ( + "EarlyFetch", + background_early_fetches_pool_size, + background_early_fetches_pool_size, + CurrentMetrics::BackgroundEarlyFetchesPoolTask, + CurrentMetrics::BackgroundEarlyFetchesPoolSize, + ProfileEvents::FetchBackgroundExecutorTaskExecuteStepMicroseconds, + ProfileEvents::FetchBackgroundExecutorTaskCancelMicroseconds, + ProfileEvents::FetchBackgroundExecutorTaskResetMicroseconds, + ProfileEvents::FetchBackgroundExecutorWaitMicroseconds + ); + LOG_INFO(shared->log, "Initialized background executor for early fetches with num_threads={}, num_tasks={}", background_early_fetches_pool_size, background_early_fetches_pool_size); + shared->common_executor = std::make_shared ( "Common", @@ -6651,6 +6671,12 @@ OrdinaryBackgroundExecutorPtr Context::getFetchesExecutor() const return shared->fetch_executor; } +OrdinaryBackgroundExecutorPtr Context::getEarlyFetchesExecutor() const +{ + SharedLockGuard lock(shared->background_executors_mutex); + return shared->early_fetch_executor; +} + OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const { SharedLockGuard lock(shared->background_executors_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f11fdbfd53b9..83cf5185cf1f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1553,6 +1553,7 @@ class Context: public ContextData, public std::enable_shared_from_this MergeMutateBackgroundExecutorPtr getMergeMutateExecutor() const; OrdinaryBackgroundExecutorPtr getMovesExecutor() const; OrdinaryBackgroundExecutorPtr getFetchesExecutor() const; + OrdinaryBackgroundExecutorPtr getEarlyFetchesExecutor() const; OrdinaryBackgroundExecutorPtr getCommonExecutor() const; IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp index 90244a3d6a35..f57ba6452b20 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp @@ -80,6 +80,14 @@ bool BackgroundJobsAssignee::scheduleFetchTask(ExecutableTaskPtr fetch_task) } +bool BackgroundJobsAssignee::scheduleEarlyFetchTask(ExecutableTaskPtr fetch_task) +{ + bool res = getContext()->getEarlyFetchesExecutor()->trySchedule(fetch_task); + res ? trigger() : postpone(); + return res; +} + + bool BackgroundJobsAssignee::scheduleMoveTask(ExecutableTaskPtr move_task) { bool res = getContext()->getMovesExecutor()->trySchedule(move_task); diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.h b/src/Storages/MergeTree/BackgroundJobsAssignee.h index a488ea23a871..169b995e1630 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.h +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.h @@ -52,6 +52,7 @@ class BackgroundJobsAssignee : public WithContext bool scheduleMergeMutateTask(ExecutableTaskPtr merge_task); bool scheduleFetchTask(ExecutableTaskPtr fetch_task); + bool scheduleEarlyFetchTask(ExecutableTaskPtr fetch_task); bool scheduleMoveTask(ExecutableTaskPtr move_task); bool scheduleCommonTask(ExecutableTaskPtr common_task, bool need_trigger); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 473729bb766a..054811073920 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -210,6 +210,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool allow_experimental_reverse_key; extern const MergeTreeSettingsBool allow_nullable_key; extern const MergeTreeSettingsBool allow_remote_fs_zero_copy_replication; + extern const MergeTreeSettingsBool use_early_fetch_pool; extern const MergeTreeSettingsBool allow_suspicious_indices; extern const MergeTreeSettingsBool allow_summing_columns_in_partition_or_order_key; extern const MergeTreeSettingsBool allow_coalescing_columns_in_partition_or_order_key; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index d6ac882a8a78..5207f3aee185 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -1657,6 +1657,10 @@ namespace ErrorCodes DECLARE(Bool, allow_remote_fs_zero_copy_replication, false, R"( Don't use this setting in production, because it is not ready. )", EXPERIMENTAL) \ + DECLARE(Bool, use_early_fetch_pool, true, R"( + Use separate thread pool for early fetches (initial sync) to prevent blocking normal replication. + When enabled, fetches with empty source_replica use the early fetches pool instead of the normal fetches pool. + )", 0) \ DECLARE(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", R"( ZooKeeper path for zero-copy table-independent info. )", EXPERIMENTAL) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7af04f5188c0..4554488aeccc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -149,6 +149,7 @@ namespace ProfileEvents namespace CurrentMetrics { extern const Metric BackgroundFetchesPoolTask; + extern const Metric BackgroundEarlyFetchesPoolTask; extern const Metric ReadonlyReplica; } @@ -190,6 +191,7 @@ namespace MergeTreeSetting { extern const MergeTreeSettingsBool allow_experimental_replacing_merge_with_cleanup; extern const MergeTreeSettingsBool allow_remote_fs_zero_copy_replication; + extern const MergeTreeSettingsBool use_early_fetch_pool; extern const MergeTreeSettingsBool always_use_copy_instead_of_hardlinks; extern const MergeTreeSettingsBool assign_part_uuids; extern const MergeTreeSettingsDeduplicateMergeProjectionMode deduplicate_merge_projection_mode; @@ -4034,11 +4036,22 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne /// Depending on entry type execute in fetches (small) pool or big merge_mutate pool if (job_type == LogEntry::GET_PART || job_type == LogEntry::ATTACH_PART) { - assignee.scheduleFetchTask(std::make_shared( + auto fetch_task = std::make_shared( [this, selected_entry] () mutable { return processQueueEntry(selected_entry); - }, common_assignee_trigger, getStorageID())); + }, common_assignee_trigger, getStorageID()); + + /// Route to early fetches pool if source_replica is empty (initial sync) and setting is enabled + /// Otherwise use normal fetches pool (ongoing replication) + if (selected_entry->log_entry->source_replica == "" && (*getSettings())[MergeTreeSetting::use_early_fetch_pool]) + { + assignee.scheduleEarlyFetchTask(fetch_task); + } + else + { + assignee.scheduleFetchTask(fetch_task); + } return true; } if (job_type == LogEntry::MERGE_PARTS) @@ -4090,12 +4103,30 @@ bool StorageReplicatedMergeTree::canExecuteFetch(const ReplicatedMergeTreeLogEnt return false; } - auto replicated_fetches_pool_size = getContext()->getFetchesExecutor()->getMaxTasksCount(); - size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundFetchesPoolTask].load(std::memory_order_relaxed); - if (busy_threads_in_pool >= replicated_fetches_pool_size) + /// Check appropriate pool based on source_replica and setting + /// Empty source_replica means initial sync (early fetch), otherwise normal fetch + if (entry.source_replica == "" && (*getSettings())[MergeTreeSetting::use_early_fetch_pool]) { - disable_reason = fmt::format("Not executing fetch of part {} because {} fetches already executing, max {}.", entry.new_part_name, busy_threads_in_pool, replicated_fetches_pool_size); - return false; + auto replicated_early_fetches_pool_size = getContext()->getEarlyFetchesExecutor()->getMaxTasksCount(); + size_t busy_early_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundEarlyFetchesPoolTask].load( + std::memory_order_relaxed); + if (busy_early_threads_in_pool >= replicated_early_fetches_pool_size) + { + disable_reason = fmt::format( + "Not executing fetch of part {} because {} early fetches already executing, max {}.", + entry.new_part_name, busy_early_threads_in_pool, replicated_early_fetches_pool_size); + return false; + } + } + else + { + auto replicated_fetches_pool_size = getContext()->getFetchesExecutor()->getMaxTasksCount(); + size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundFetchesPoolTask].load(std::memory_order_relaxed); + if (busy_threads_in_pool >= replicated_fetches_pool_size) + { + disable_reason = fmt::format("Not executing fetch of part {} because {} fetches already executing, max {}.", entry.new_part_name, busy_threads_in_pool, replicated_fetches_pool_size); + return false; + } } if (replicated_fetches_throttler->isThrottling()) From c6a08f02640daf7e483223fec346f885083eec5a Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Tue, 6 Jan 2026 16:21:30 +0100 Subject: [PATCH 48/49] Add per-server override for max bytes to merge/mutate During a maintenance upgrade, we do not wait for the completion of merges and mutations since they are not required to make sure we have all the data from the previous nodes. We do wait for `GET_PART` and similar tasks, since we need the new nodes to get parts from the old nodes. This is implemented using `SYSTEM SYNC REPLICA ... LIGHTWEIGHT`. Some merges and mutations prevent the execution of `GET_PART` tasks that overlap the range of the merge or the mutation. So, even if we are not waiting for `MERGE_PARTS` tasks, these tasks can slow down the completion of a maintenance upgrade. We need to execute `MERGE_PARTS` tasks to avoid having too many parts in the same partition or table. We also need to execute the TTL delete rules and ensure disk usage does not grow too much (TTL deletes are implemented as a subtype of the merge tasks). A possible tradeoff is to only execute merge tasks if they are not too large, the small merges are the ones that keep the number of parts low when the are many small INSERT queries. We can't use the normal merge tree settings like `max_bytes_to_merge_at_min/max_space_in_pool` because they would be applied with `ALTER TABLE` which would step onto user-managed objects and become persisted as part of the table definition. They are also replicated, we can't set them only on new nodes. The patch implements per-server overrides, that we can use to limit the size of merge and mutate tasks, both when a node creates new tasks and when it decides which task to execute. This is only usable with a `LIGHTWEIGHT` sync: if we don't execute some tasks, we also need to not wait for them, or we would wait forever. `SYNC REPLICA` with the `LIGHTWEIGHT` flag allows us to not wait for these tasks. This commit was applied from the patch file 0078-Global_merge_and_mutate_override.patch Co-authored-by: Kevin Michel --- programs/server/Server.cpp | 5 ++++ src/Core/ServerSettings.cpp | 2 ++ src/Interpreters/Context.cpp | 28 ++++++++++++++++++- src/Interpreters/Context.h | 6 ++++ .../Compaction/CompactionStatistics.cpp | 21 +++++++++++--- 5 files changed, 57 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d279e7560a99..79f31671253c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -331,6 +331,8 @@ namespace ServerSetting extern const ServerSettingsUInt64 page_cache_max_size; extern const ServerSettingsDouble page_cache_free_memory_ratio; extern const ServerSettingsUInt64 page_cache_shards; + extern const ServerSettingsUInt64 max_bytes_to_merge_override; + extern const ServerSettingsUInt64 max_bytes_to_mutate_override; extern const ServerSettingsUInt64 os_cpu_busy_time_threshold; extern const ServerSettingsFloat min_os_cpu_wait_time_ratio_to_drop_connection; extern const ServerSettingsFloat max_os_cpu_wait_time_ratio_to_drop_connection; @@ -2011,6 +2013,9 @@ try global_context->setMaxPendingMutationsExecutionTimeToWarn(new_server_settings[ServerSetting::max_pending_mutations_execution_time_to_warn]); global_context->getAccessControl().setAllowTierSettings(new_server_settings[ServerSetting::allow_feature_tier]); + global_context->setMaxBytesToMergeOverride(new_server_settings[ServerSetting::max_bytes_to_merge_override]); + global_context->setMaxBytesToMutateOverride(new_server_settings[ServerSetting::max_bytes_to_mutate_override]); + global_context->setS3QueueDisableStreaming(new_server_settings[ServerSetting::s3queue_disable_streaming]); global_context->setOSCPUOverloadSettings(new_server_settings[ServerSetting::min_os_cpu_wait_time_ratio_to_drop_connection], new_server_settings[ServerSetting::max_os_cpu_wait_time_ratio_to_drop_connection]); diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 07ca3a20fb65..266d2da6e741 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -292,6 +292,8 @@ namespace DB The maximum memory consumption of the server is further restricted by setting `max_server_memory_usage`. ::: )", 0) \ + DECLARE(UInt64, max_bytes_to_merge_override, 0, R"(Maximum total size of parts to merge, global override. Zero means unlimited.)", 0) \ + DECLARE(UInt64, max_bytes_to_mutate_override, 0, R"(Maximum size of part to mutate, global override. Zero means unlimited.)", 0) \ DECLARE(UInt64, merges_mutations_memory_usage_soft_limit, 0, R"( Sets the limit on how much RAM is allowed to use for performing merge and mutation operations. If ClickHouse reaches the limit set, it won't schedule any new background merge or mutation operations but will continue to execute already scheduled tasks. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a87817f8a91a..2b1a3b535488 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -601,6 +601,8 @@ struct ContextSharedPart : boost::noncopyable std::atomic_size_t max_part_num_to_warn = 100000lu; // these variables are used in inserting warning message into system.warning table based on asynchronous metrics size_t max_pending_mutations_to_warn = 500lu; + UInt64 max_bytes_to_merge_override = 0; + UInt64 max_bytes_to_mutate_override = 0; size_t max_pending_mutations_execution_time_to_warn = 86400lu; /// Only for system.server_settings, actually value stored in reloader itself std::atomic_size_t config_reload_interval_ms = ConfigReloader::DEFAULT_RELOAD_INTERVAL.count(); @@ -4806,6 +4808,18 @@ size_t Context::getMaxPendingMutationsToWarn() const return shared->max_pending_mutations_to_warn; } +UInt64 Context::getMaxBytesToMergeOverride() const +{ + SharedLockGuard lock(shared->mutex); + return shared->max_bytes_to_merge_override; +} + +UInt64 Context::getMaxBytesToMutateOverride() const +{ + SharedLockGuard lock(shared->mutex); + return shared->max_bytes_to_mutate_override; +} + size_t Context::getMaxPendingMutationsExecutionTimeToWarn() const { SharedLockGuard lock(shared->mutex); @@ -4844,10 +4858,22 @@ size_t Context::getMaxDatabaseNumToWarn() const void Context::setMaxPendingMutationsToWarn(size_t max_pending_mutations_to_warn) { - SharedLockGuard lock(shared->mutex); + std::lock_guard lock(shared->mutex); shared->max_pending_mutations_to_warn = max_pending_mutations_to_warn; } +void Context::setMaxBytesToMergeOverride(UInt64 max_bytes_to_merge_override) +{ + std::lock_guard lock(shared->mutex); + shared->max_bytes_to_merge_override = max_bytes_to_merge_override; +} + +void Context::setMaxBytesToMutateOverride(UInt64 max_bytes_to_mutate_override) +{ + std::lock_guard lock(shared->mutex); + shared->max_bytes_to_mutate_override = max_bytes_to_mutate_override; +} + void Context::setMaxPendingMutationsExecutionTimeToWarn(size_t max_pending_mutations_execution_time_to_warn) { SharedLockGuard lock(shared->mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 83cf5185cf1f..2c17bbed1275 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1082,6 +1082,9 @@ class Context: public ContextData, public std::enable_shared_from_this size_t getMaxPendingMutationsToWarn() const; size_t getMaxPendingMutationsExecutionTimeToWarn() const; + UInt64 getMaxBytesToMergeOverride() const; + UInt64 getMaxBytesToMutateOverride() const; + void setMaxTableNumToWarn(size_t max_table_to_warn); void setMaxViewNumToWarn(size_t max_view_to_warn); void setMaxDictionaryNumToWarn(size_t max_dictionary_to_warn); @@ -1091,6 +1094,9 @@ class Context: public ContextData, public std::enable_shared_from_this void setMaxPendingMutationsToWarn(size_t max_pending_mutations_to_warn); void setMaxPendingMutationsExecutionTimeToWarn(size_t max_pending_mutations_execution_time_to_warn); + void setMaxBytesToMergeOverride(UInt64 max_bytes_to_merge_override); + void setMaxBytesToMutateOverride(UInt64 max_bytes_to_mutate_override); + double getMinOSCPUWaitTimeRatioToDropConnection() const; double getMaxOSCPUWaitTimeRatioToDropConnection() const; void setOSCPUOverloadSettings(double min_os_cpu_wait_time_ratio_to_drop_connection, double max_os_cpu_wait_time_ratio_to_drop_connection); diff --git a/src/Storages/MergeTree/Compaction/CompactionStatistics.cpp b/src/Storages/MergeTree/Compaction/CompactionStatistics.cpp index d6d57569b2f0..dd710542b552 100644 --- a/src/Storages/MergeTree/Compaction/CompactionStatistics.cpp +++ b/src/Storages/MergeTree/Compaction/CompactionStatistics.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include @@ -80,13 +81,19 @@ UInt64 getMaxSourcePartsSizeForMerge(const MergeTreeData & data) UInt64 getMaxSourcePartsSizeForMerge(const MergeTreeData & data, size_t max_count, size_t scheduled_tasks_count) { const auto data_settings = data.getSettings(); - return getMaxSourcePartsSizeForMerge( + UInt64 max_size = getMaxSourcePartsSizeForMerge( /*max_count=*/max_count, /*scheduled_tasks_count=*/scheduled_tasks_count, /*max_unreserved_free_space*/data.getStoragePolicy()->getMaxUnreservedFreeSpace(), /*size_lowering_threshold=*/(*data_settings)[MergeTreeSetting::number_of_free_entries_in_pool_to_lower_max_size_of_merge], /*size_limit_at_min_pool_space=*/(*data_settings)[MergeTreeSetting::max_bytes_to_merge_at_min_space_in_pool], /*size_limit_at_max_pool_space=*/(*data_settings)[MergeTreeSetting::max_bytes_to_merge_at_max_space_in_pool]); + + UInt64 max_bytes_to_merge_override = data.getContext()->getMaxBytesToMergeOverride(); + if (max_bytes_to_merge_override != 0) + max_size = std::min(max_size, max_bytes_to_merge_override); + + return max_size; } UInt64 getMaxSourcePartsSizeForMerge( @@ -151,13 +158,19 @@ UInt64 getMaxSourcePartSizeForMutation(const MergeTreeData & data, String * out_ /// Allow mutations only if there are enough threads, otherwise, leave free threads for merges. size_t number_of_free_entries_in_pool_to_execute_mutation = (*data_settings)[MergeTreeSetting::number_of_free_entries_in_pool_to_execute_mutation]; + UInt64 max_size = 0; if (occupied <= 1 || max_tasks_count - occupied >= number_of_free_entries_in_pool_to_execute_mutation) - return static_cast(disk_space / DISK_USAGE_COEFFICIENT_TO_RESERVE); + { + max_size = static_cast(disk_space / DISK_USAGE_COEFFICIENT_TO_RESERVE); + UInt64 max_bytes_to_mutate_override = data.getContext()->getMaxBytesToMutateOverride(); + if (max_bytes_to_mutate_override != 0) + max_size = std::min(max_size, max_bytes_to_mutate_override); + } - if (out_log_comment) + if (out_log_comment && max_size == 0) *out_log_comment = fmt::format("max_tasks_count ({}) - occupied ({}) >= number_of_free_entries_in_pool_to_execute_mutation ({})", max_tasks_count, occupied, number_of_free_entries_in_pool_to_execute_mutation); - return 0; + return max_size; } } From 512d6a8a5d7c6614f3fdfbf78a6083b5c94ef9ed Mon Sep 17 00:00:00 2001 From: Tilman Moeller Date: Wed, 7 Jan 2026 11:24:57 +0100 Subject: [PATCH 49/49] Zero copy fixes 1. Fix sharding race condition with zero copy. Locks are shared between parts with the same name on different shards. This can lead to a race condition when committing the same part on different shards for the znode `/clickhouse/zero_copy/zero_copy_s3/{uuid}/all_0_0_0`. The fix is to create this znode before committing the part and using createIfNotExists style operations. Especially prevalent on SYSTEM RESTORE REPLICA on the first partition. 2. Remove zero copy replication error when dropping table - occurs when Astacus has left files in the table directory after a failed restore. Changes: - Added zero copy lock znode pre-creation in ReplicatedMergeTreeSink.cpp before part commit using createAncestors() and createIfNotExists() - Removed error check in MergeTreeData::dropAllData() that blocked table drop when directory was not empty - Added extern declaration for allow_remote_fs_zero_copy_replication in ReplicatedMergeTreeSink.cpp The zero copy lock pre-creation prevents race conditions when multiple shards commit parts with the same name simultaneously. The createIfNotExists() operation is idempotent, allowing safe concurrent execution across shards. Removing the strict validation in dropAllData() allows tables to be dropped even when backup/restore tools leave leftover files, as removeRecursive() will clean them up anyway. Co-authored-by: Joe Lynch --- src/Storages/MergeTree/MergeTreeData.cpp | 13 ------------- .../MergeTree/ReplicatedMergeTreeSink.cpp | 15 +++++++++++++++ 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 054811073920..1caf09a0727e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3832,19 +3832,6 @@ void MergeTreeData::dropAllData() try { - if (!isSharedStorage() && !disk->isDirectoryEmpty(relative_data_path) && - supportsReplication() && disk->supportZeroCopyReplication() - && (*settings_ptr)[MergeTreeSetting::allow_remote_fs_zero_copy_replication]) - { - std::vector files_left; - disk->listFiles(relative_data_path, files_left); - - throw Exception( - ErrorCodes::ZERO_COPY_REPLICATION_ERROR, - "Directory {} with table {} not empty (files [{}]) after drop. Will not drop.", - relative_data_path, getStorageID().getNameForLogs(), fmt::join(files_left, ", ")); - } - LOG_INFO(log, "dropAllData: removing table directory recursive to cleanup garbage"); disk->removeRecursive(relative_data_path); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 56bd38edf7c7..e4e80cfa66af 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -44,6 +44,7 @@ namespace Setting namespace MergeTreeSetting { extern const MergeTreeSettingsMilliseconds sleep_before_commit_local_part_in_replicated_table_ms; + extern const MergeTreeSettingsBool allow_remote_fs_zero_copy_replication; } namespace FailPoints @@ -902,6 +903,20 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: part->info.max_block = block_number; part->setName(part->getNewName(part->info)); + + const auto storage_settings = storage.getSettings(); + if ((*storage_settings)[MergeTreeSetting::allow_remote_fs_zero_copy_replication] + && part->getDataPartStorage().supportZeroCopyReplication()) { + const auto zero_copy_lock_part_paths = storage.getZeroCopyPartPath( + *storage_settings, part->getDataPartStorage().getDiskType(), storage.getTableSharedID(), + part->name, storage.zookeeper_path, storage.getContext()); + for (const auto & path : zero_copy_lock_part_paths) + { + zookeeper->createAncestors(path); + zookeeper->createIfNotExists(path, ""); + } + } + retry_context.actual_part_name = part->name; /// Prepare transaction to ZooKeeper