diff --git a/.gitmodules b/.gitmodules index 1c7304defc3c..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 @@ -87,7 +88,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 @@ -214,7 +216,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/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") diff --git a/contrib/aws b/contrib/aws index a86b913abc27..30c8374334bb 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit a86b913abc2795ee23941b24dd51e862214ec6b0 +Subproject commit 30c8374334bbfa1c76f336e9827f178337f27703 diff --git a/contrib/azure b/contrib/azure index 0f7a2013f7d7..ceb5b8534eb6 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 0f7a2013f7d79058047fc4bd35e94d20578c0d2b +Subproject commit ceb5b8534eb67e8490ccf28cab228b6882882e23 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}" ) 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/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/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/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 70818a25af16..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; @@ -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..f55401c0d6e5 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); @@ -264,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/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/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/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/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/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 51cf22a12eee..f12ba71bcd6a 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -146,11 +146,13 @@ 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, 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/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/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/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index bb5f62c162a6..561d061d52de 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -171,8 +171,6 @@ AsynchronousMetrics::AsynchronousMetrics( openSensors(); openBlockDevices(); - openEDAC(); - openSensorsChips(); #endif } @@ -994,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." }; @@ -1002,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/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 1b4071cbb5dc..1940021f96d6 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -14,10 +14,13 @@ 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") \ 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/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/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/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/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) \ 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); } } 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/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/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 0973fdc045c9..6811e97c2552 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -122,11 +122,13 @@ 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 = */ {}, /* put_request_throttler = */ {}, - new_uri.uri.getScheme()); + new_uri.uri.getScheme(), + /* signature_delegation_url = */ ""); client_configuration.endpointOverride = new_uri.endpoint; 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/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 004f8a16098c..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. @@ -812,6 +814,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) \ @@ -1027,13 +1030,16 @@ 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"( 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) \ @@ -1162,6 +1168,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", @@ -1348,6 +1355,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/Core/Settings.cpp b/src/Core/Settings.cpp index 903d984dee49..8aec7c12ce85 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 @@ -2914,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"( @@ -3833,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..db926fc89f1d 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -106,6 +107,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}, @@ -151,6 +165,110 @@ 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(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}, {"force_enable", ShortCircuitFunctionEvaluation::FORCE_ENABLE}, @@ -307,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 18873a0790ae..6828f1533ccd 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -266,6 +266,48 @@ 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) + +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) enum class TransactionsWaitCSNMode : uint8_t @@ -289,6 +331,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, @@ -351,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/Databases/DataLake/GlueCatalog.cpp b/src/Databases/DataLake/GlueCatalog.cpp index c0233f362ec7..110320690f28 100644 --- a/src/Databases/DataLake/GlueCatalog.cpp +++ b/src/Databases/DataLake/GlueCatalog.cpp @@ -121,10 +121,13 @@ 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, - /* 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/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index f99884741315..fe2d796727cd 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -55,6 +55,7 @@ #include #include #include +#include "Storages/PartitionCommands.h" namespace DB @@ -123,7 +124,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) @@ -142,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) @@ -188,6 +192,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); @@ -386,7 +404,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); } @@ -1387,6 +1407,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. @@ -2264,11 +2285,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 (...) @@ -2314,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. @@ -2325,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); @@ -2341,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 0100723e28c5..b0cb58a08e02 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -46,12 +46,14 @@ 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); ~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. @@ -80,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; @@ -186,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/Databases/DatabaseReplicatedSettings.cpp b/src/Databases/DatabaseReplicatedSettings.cpp index c1a737af49af..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) @@ -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; 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..3376d3e63c96 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 @@ -87,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; @@ -121,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('\\')) { @@ -152,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 { @@ -163,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 &) { @@ -183,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; @@ -264,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); } @@ -271,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); @@ -305,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); @@ -362,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 (...) { @@ -380,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; } @@ -413,8 +512,6 @@ void DatabasePostgreSQL::removeOutdatedTables() else ++iter; } - - cleaner_task->scheduleAfter(cleaner_reschedule_ms); } @@ -443,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; } @@ -591,7 +723,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/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; 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/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..d0b2a0797782 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -29,12 +29,15 @@ 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 { extern const int SUPPORT_IS_DISABLED; extern const int BAD_ARGUMENTS; + extern const int UNSUPPORTED_METHOD; } static const ValidateKeysMultiset dictionary_allowed_keys = { @@ -186,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) @@ -217,117 +207,40 @@ 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], settings[Setting::postgresql_connection_pool_auto_close_connection], settings[Setting::postgresql_connection_attempt_timeout], - bg_reconnect); + static_cast(settings[Setting::postgresql_connection_pool_ssl_mode]), + static_cast(settings[Setting::postgresql_connection_pool_ssl_root_cert]), + 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/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); } 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.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(); } 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/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 8d3b3696be12..b4f705171dab 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 @@ -95,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)) @@ -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; @@ -390,6 +408,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); } @@ -495,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); @@ -584,6 +607,16 @@ 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"); + } + + 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 e0bd961b2f6f..a75ac00c75f0 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -60,6 +60,10 @@ 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; + + std::optional account_name; + std::optional signature_delegation_url; #endif }; @@ -141,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; @@ -160,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/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/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( diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index e9c4fe2df80a..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 @@ -122,11 +123,13 @@ 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, 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/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/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/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 c46d1456c417..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 @@ -1219,11 +1271,13 @@ 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, const ThrottlerPtr & put_request_throttler, - const String & protocol) + const String & protocol, + const String & signature_delegation_url) { auto context = Context::getGlobalContextInstance(); chassert(context); @@ -1241,11 +1295,13 @@ 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], 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 16457c7a511f..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; @@ -358,11 +357,13 @@ 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, 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 aed33b800c87..8920f5d63bf3 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -741,10 +741,13 @@ 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, - 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)); } @@ -759,10 +762,13 @@ 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, - 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)); } @@ -812,11 +818,13 @@ 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, 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 7b36c5195a34..f6d4b3b6e99c 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,11 +104,13 @@ 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_, 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_) @@ -117,10 +120,12 @@ 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_) , put_request_throttler(put_request_throttler_) + , signature_delegation_url(signature_delegation_url_) , s3_use_adaptive_timeouts(s3_use_adaptive_timeouts_) , error_report(error_report_) { @@ -189,6 +194,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 +521,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 +536,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 +814,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..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 { @@ -59,6 +60,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; @@ -69,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; @@ -93,11 +96,13 @@ 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_, 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. @@ -220,6 +225,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..d66cbdb6fb70 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -140,11 +140,13 @@ 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 = */ {}, /* put_request_throttler = */ {}, - uri.uri.getScheme()); + uri.uri.getScheme(), + /* signature_delegation_url = */ ""); client_configuration.endpointOverride = uri.endpoint; @@ -416,11 +418,13 @@ 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 = */ {}, /* 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/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..13a6bde4b107 100644 --- a/src/IO/tests/gtest_readbuffer_s3.cpp +++ b/src/IO/tests/gtest_readbuffer_s3.cpp @@ -60,13 +60,15 @@ struct ClientFake : DB::S3::Client true, true, true, + /* ca_path = */ std::optional(), false, {}, {}, {}, - "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 444c19457dd8..8d8a7997e39a 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -248,10 +248,13 @@ 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 = */ {}, - /* put_request_throttler = */ {}); + /* put_request_throttler = */ {}, + /* protocol = */ "https", + /* signature_delegation_url = */ ""); } void setInjectionModel(std::shared_ptr injections_) 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()) { 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 2b5381e96400..c73a0abb9b55 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -1,18 +1,27 @@ #include #include +#include #include #include #include #include +#include +#include #include #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 +29,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 +485,54 @@ 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, " + "CHECK, " + "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); @@ -477,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/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, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e1a1e3b677c1..2b1a3b535488 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -208,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; @@ -236,6 +239,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; @@ -331,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; @@ -384,6 +389,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; @@ -595,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(); @@ -616,6 +624,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; @@ -623,6 +635,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 @@ -861,6 +874,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()); @@ -1759,6 +1773,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(); @@ -4755,25 +4775,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 @@ -4782,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); @@ -4820,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); @@ -5009,6 +5059,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) { @@ -6520,6 +6594,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 @@ -6567,6 +6642,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", @@ -6608,6 +6697,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 626705099e22..2c17bbed1275 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; @@ -1080,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); @@ -1089,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); @@ -1106,6 +1114,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; @@ -1397,6 +1407,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; @@ -1545,6 +1559,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/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/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/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/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/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/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/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..063b37fc737c 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") \ @@ -393,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") \ 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 TypePromotionsendRequest(request); Poco::Net::HTTPResponse response; 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(); 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); } } diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 23b73f3d823f..3410821cf2fb 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -224,7 +224,7 @@ void ReadFromCluster::initializePipeline(QueryPipelineBuilder & pipeline, const auto remote_query_executor = std::make_shared( std::vector{try_results.front()}, - query_to_send->formatWithSecretsOneLine(), + query_to_send->formatWithSecretsOneLine(false), getOutputHeader(), new_context, /*throttler=*/nullptr, diff --git a/src/Storages/Kafka/KafkaConfigLoader.cpp b/src/Storages/Kafka/KafkaConfigLoader.cpp index e4712c4d5f85..2f3131f6873e 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")) @@ -458,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 @@ -498,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 a007218723c5..775a18c2445b 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 @@ -38,12 +47,21 @@ struct KafkaConfigLoader size_t consumer_number; String client_id; size_t max_block_size; + String auto_offset_reset; }; struct ProducerConfigParams : public LoadConfigParams { 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 35fa5f0e883f..79be9e6ee85b 100644 --- a/src/Storages/Kafka/KafkaSettings.cpp +++ b/src/Storages/Kafka/KafkaSettings.cpp @@ -24,6 +24,8 @@ 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(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 */ \ @@ -43,10 +45,23 @@ 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) \ + 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 a7362c8ff44c..075c4497eb53 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -33,6 +33,11 @@ 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, KafkaCompressionCodec) \ + 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..2af7f959d411 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -39,6 +39,7 @@ #include #include +#include #include #include @@ -88,8 +89,26 @@ 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 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; + 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; + 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 @@ -180,6 +199,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) @@ -461,22 +481,47 @@ 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, 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); } 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}; + 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); } @@ -534,8 +579,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/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/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index c55da9c24790..660543a3b277 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -104,8 +105,25 @@ 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 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; + 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; + 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; @@ -457,13 +475,22 @@ 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, 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"); @@ -475,16 +502,33 @@ 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}; + 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); } 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..916dc6fa35ac 100644 --- a/src/Storages/Kafka/StorageKafkaUtils.cpp +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -82,6 +82,8 @@ 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; + extern const KafkaSettingsDateTimeInputFormat kafka_date_time_input_format; } using namespace std::chrono_literals; @@ -205,10 +207,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) { @@ -524,6 +522,13 @@ 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); + } + + result.emplace_back("date_time_input_format", kafka_settings[KafkaSetting::kafka_date_time_input_format].toString()); return result; } 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/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; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d1fea772e267..1caf09a0727e 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; @@ -206,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; @@ -262,6 +267,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; @@ -2146,18 +2155,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()); } } @@ -3815,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); } @@ -5397,7 +5401,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 +5468,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 +5534,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..5207f3aee185 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 @@ -1653,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/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..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 @@ -1192,7 +1207,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/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/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); 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); 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/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); 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..bed95eaabed5 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 @@ -529,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"}); + 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()) @@ -560,6 +571,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 +640,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..c7f9c54d05c7 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -3,8 +3,10 @@ #include "config.h" #if USE_LIBPQXX +#include #include #include +#include namespace Poco { @@ -56,6 +58,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; @@ -68,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, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3a4374a6c9dc..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; } @@ -183,12 +184,14 @@ 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 { 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; @@ -416,6 +419,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 +1414,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); } @@ -1640,9 +1650,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. @@ -4011,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) @@ -4038,6 +4074,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 { @@ -4047,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()) @@ -5634,6 +5708,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 +5781,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..e5ad246e76aa 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -261,9 +262,15 @@ 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; + // 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 +469,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 +523,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; 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); } 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]) 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); 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])); } }