From f55e4f8f289e135ab65ca75fe6bb354b8cf00988 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 14 Mar 2024 09:51:02 +0100 Subject: [PATCH 1/9] Add test. --- .../test_dictionaries_replace/__init__.py | 0 .../configs/allow_database_types.xml | 7 + .../configs/no_lazy_load.xml | 3 + .../configs/remote_servers.xml | 16 +++ .../test_dictionaries_replace/test.py | 136 ++++++++++++++++++ 5 files changed, 162 insertions(+) create mode 100644 tests/integration/test_dictionaries_replace/__init__.py create mode 100644 tests/integration/test_dictionaries_replace/configs/allow_database_types.xml create mode 100644 tests/integration/test_dictionaries_replace/configs/no_lazy_load.xml create mode 100644 tests/integration/test_dictionaries_replace/configs/remote_servers.xml create mode 100644 tests/integration/test_dictionaries_replace/test.py diff --git a/tests/integration/test_dictionaries_replace/__init__.py b/tests/integration/test_dictionaries_replace/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_dictionaries_replace/configs/allow_database_types.xml b/tests/integration/test_dictionaries_replace/configs/allow_database_types.xml new file mode 100644 index 000000000000..0434df06457c --- /dev/null +++ b/tests/integration/test_dictionaries_replace/configs/allow_database_types.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_dictionaries_replace/configs/no_lazy_load.xml b/tests/integration/test_dictionaries_replace/configs/no_lazy_load.xml new file mode 100644 index 000000000000..aaae3e0c4c1c --- /dev/null +++ b/tests/integration/test_dictionaries_replace/configs/no_lazy_load.xml @@ -0,0 +1,3 @@ + + 0 + diff --git a/tests/integration/test_dictionaries_replace/configs/remote_servers.xml b/tests/integration/test_dictionaries_replace/configs/remote_servers.xml new file mode 100644 index 000000000000..6362032973ff --- /dev/null +++ b/tests/integration/test_dictionaries_replace/configs/remote_servers.xml @@ -0,0 +1,16 @@ + + + + + + node_ll + 9000 + + + node_no_ll + 9000 + + + + + diff --git a/tests/integration/test_dictionaries_replace/test.py b/tests/integration/test_dictionaries_replace/test.py new file mode 100644 index 000000000000..bf406f46cb15 --- /dev/null +++ b/tests/integration/test_dictionaries_replace/test.py @@ -0,0 +1,136 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + + +cluster = ClickHouseCluster(__file__) + +node_ll = cluster.add_instance( + "node_ll", + main_configs=[ + "configs/remote_servers.xml", + ], + user_configs=[ + "configs/allow_database_types.xml", + ], + macros={"replica": "node_ll", "shard": "shard"}, + with_zookeeper=True, +) + +node_no_ll = cluster.add_instance( + "node_no_ll", + main_configs=[ + "configs/no_lazy_load.xml", + "configs/remote_servers.xml", + ], + user_configs=[ + "configs/allow_database_types.xml", + ], + macros={"replica": "node_no_ll", "shard": "shard"}, + with_zookeeper=True, +) + +instances = [node_ll, node_no_ll] + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + # "sleep(1)" is here to make loading of dictionaries a bit slower for this test. + instances[0].query( + "CREATE VIEW src ON CLUSTER 'cluster' AS SELECT number AS key, number * number + 1 + sleep(1) AS value FROM numbers(10)" + ) + + # "dict_get_user" can only call dictGet(), nothing more. + instances[0].query("CREATE USER dictget_user ON CLUSTER 'cluster'") + instances[0].query( + "GRANT dictGet ON atomicdb.dict TO dictget_user ON CLUSTER 'cluster'" + ) + instances[0].query( + "GRANT dictGet ON repldb.dict TO dictget_user ON CLUSTER 'cluster'" + ) + + instances[0].query("CREATE DATABASE atomicdb ON CLUSTER 'cluster'") + instances[0].query( + "CREATE DATABASE repldb ON CLUSTER 'cluster' ENGINE=Replicated('/clickhouse/path/','{shard}','{replica}')" + ) + + yield cluster + + finally: + instances[0].query("DROP TABLE IF EXISTS src ON CLUSTER 'cluster'") + instances[0].query("DROP USER IF EXISTS dictget_user ON CLUSTER 'cluster'") + instances[0].query("DROP DATABASE IF EXISTS atomicdb ON CLUSTER 'cluster'") + instances[0].query("DROP DATABASE IF EXISTS repldb ON CLUSTER 'cluster'") + + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + instances[0].query("DROP DICTIONARY IF EXISTS dict ON CLUSTER 'cluster'") + + +def get_status(instance, dictionary_name): + return instance.query( + "SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'" + ).rstrip("\n") + + +@pytest.mark.parametrize( + "database, instance_to_create_dictionary, instances_to_check", + [ + ("atomicdb", node_ll, [node_ll]), + ("atomicdb", node_no_ll, [node_no_ll]), + ("repldb", node_ll, [node_ll, node_no_ll]), + ("repldb", node_no_ll, [node_ll, node_no_ll]), + ], +) +def test_create_or_replace(database, instance_to_create_dictionary, instances_to_check): + num_steps = 2 + dict_uuids = {} + for step in range(0, num_steps): + create_dictionary_query = f"CREATE OR REPLACE DICTIONARY {database}.dict (key Int64, value Int64) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'src' PASSWORD '' DB 'default')) LIFETIME(0) LAYOUT(FLAT())" + instance_to_create_dictionary.query(create_dictionary_query) + + for instance in instances_to_check: + if instance != instance_to_create_dictionary: + instance.query(f"SYSTEM SYNC DATABASE REPLICA {database}") + + dict_uuid = instance.query( + f"SELECT uuid FROM system.tables WHERE database='{database}' AND table='dict'" + ).strip("\n") + + dict_db, dict_name, dict_status = ( + instance.query( + f"SELECT database, name, status FROM system.dictionaries WHERE uuid='{dict_uuid}'" + ) + .strip("\n") + .split("\t") + ) + assert dict_db == database + assert dict_name == "dict" + + # "uuid" must be the same for all the dictionaries created at the same "step" and different for the dictionaries created at different steps. + if step in dict_uuids: + assert dict_uuids[step] == dict_uuid + dict_uuids[step] = dict_uuid + assert dict_uuid not in [ + dict_uuids[prev_step] for prev_step in range(0, step) + ] + + expected_dict_status = ( + ["NOT_LOADED"] if instance == node_ll else ["LOADING", "LOADED"] + ) + assert dict_status in expected_dict_status + + for instance in instances_to_check: + select_query = f"SELECT arrayJoin([0, 5, 7, 11]) as key, dictGet({database}.dict, 'value', key)" + expected_result = TSV([[0, 1], [5, 26], [7, 50], [11, 0]]) + assert instance.query(select_query) == expected_result + assert instance.query(select_query, user="dictget_user") == expected_result From 41d8621d975497f7a5b0c1fbe845a6410a2a5341 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 14 Mar 2024 23:50:01 +0100 Subject: [PATCH 2/9] Fix replacing dictionaries: StorageID is now updated immediately for every dictionary during CREATE OR REPLACE even if the dictionary is not loaded yet or loading right now. --- src/Dictionaries/IDictionary.h | 18 +++++++------ .../ExternalDictionariesLoader.cpp | 14 ++++++++++ src/Interpreters/ExternalDictionariesLoader.h | 3 +++ src/Interpreters/ExternalLoader.cpp | 21 ++++++++++++--- src/Interpreters/ExternalLoader.h | 4 +++ src/Storages/StorageDictionary.cpp | 27 ++++++++++--------- 6 files changed, 63 insertions(+), 24 deletions(-) diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index bab3e470b797..9770e93669eb 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -75,13 +75,6 @@ class IDictionary : public IExternalLoadable, public IKeyValueEntity return dictionary_id; } - void updateDictionaryName(const StorageID & new_name) const - { - std::lock_guard lock{mutex}; - assert(new_name.uuid == dictionary_id.uuid && dictionary_id.uuid != UUIDHelpers::Nil); - dictionary_id = new_name; - } - std::string getLoadableName() const final { std::lock_guard lock{mutex}; @@ -452,9 +445,18 @@ class IDictionary : public IExternalLoadable, public IKeyValueEntity return sample_block; } + /// Internally called by ExternalDictionariesLoader. + /// In order to update the dictionary ID change its configuration first and then call ExternalDictionariesLoader::reloadConfig(). + void updateDictionaryID(const StorageID & new_dictionary_id) + { + std::lock_guard lock{mutex}; + assert((new_dictionary_id.uuid == dictionary_id.uuid) && (dictionary_id.uuid != UUIDHelpers::Nil)); + dictionary_id = new_dictionary_id; + } + private: mutable std::mutex mutex; - mutable StorageID dictionary_id TSA_GUARDED_BY(mutex); + StorageID dictionary_id TSA_GUARDED_BY(mutex); String dictionary_comment TSA_GUARDED_BY(mutex); }; diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 74984de00643..760f94d18c24 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -41,6 +41,20 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( return DictionaryFactory::instance().create(name, config, key_in_config, getContext(), created_from_ddl); } +void ExternalDictionariesLoader::updateObjectFromConfigWithoutReloading(IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const +{ + IDictionary & dict = static_cast(object); + + auto new_dictionary_id = StorageID::fromDictionaryConfig(config, key_in_config); + auto old_dictionary_id = dict.getDictionaryID(); + if ((new_dictionary_id.table_name != old_dictionary_id.table_name) || (new_dictionary_id.database_name != old_dictionary_id.database_name)) + { + /// We can update the dictionary ID without reloading only if it's in the atomic database. + if ((new_dictionary_id.uuid == old_dictionary_id.uuid) && (new_dictionary_id.uuid != UUIDHelpers::Nil)) + dict.updateDictionaryID(new_dictionary_id); + } +} + ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 5f27d695c8ba..3c226f998be2 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -43,6 +43,9 @@ class ExternalDictionariesLoader : public ExternalLoader, WithContext LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config, const std::string & repository_name) const override; + void updateObjectFromConfigWithoutReloading( + IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const override; + std::string resolveDictionaryName(const std::string & dictionary_name, const std::string & current_database_name) const; /// Try convert qualified dictionary name to persistent UUID diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 36664cbd06fb..ec4290544754 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -87,6 +87,8 @@ namespace lock = std::unique_lock(mutex); } }; + + using UpdateObjectFromConfigWithoutReloadingFunction = std::function; } @@ -377,7 +379,7 @@ class ExternalLoader::LoadablesConfigReader : private boost::noncopyable } const String type_name; - LoggerPtr log; + const LoggerPtr log; std::mutex mutex; ExternalLoaderConfigSettings settings; @@ -400,9 +402,11 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable LoadingDispatcher( const CreateObjectFunction & create_object_function_, + const UpdateObjectFromConfigWithoutReloadingFunction & update_object_from_config_without_reloading_, const String & type_name_, LoggerPtr log_) : create_object(create_object_function_) + , update_object_from_config_without_reloading(update_object_from_config_without_reloading_) , type_name(type_name_) , log(log_) { @@ -457,10 +461,15 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable else { const auto & new_config = new_config_it->second; - bool config_is_same = isSameConfiguration(*info.config->config, info.config->key_in_config, *new_config->config, new_config->key_in_config); + auto previous_config = info.config; info.config = new_config; - if (!config_is_same) + + bool config_changed = !isSameConfiguration(*previous_config->config, previous_config->key_in_config, *new_config->config, new_config->key_in_config); + if (config_changed) { + if (info.object) + update_object_from_config_without_reloading(*info.object, *new_config->config, new_config->key_in_config); + if (info.triedToLoad()) { /// The object has been tried to load before, so it is currently in use or was in use @@ -1117,7 +1126,10 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable } if (new_object) + { + update_object_from_config_without_reloading(*new_object, *info->config->config, info->config->key_in_config); info->object = new_object; + } info->exception = new_exception; info->error_count = error_count; @@ -1192,6 +1204,7 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable } const CreateObjectFunction create_object; + const UpdateObjectFromConfigWithoutReloadingFunction update_object_from_config_without_reloading; const String type_name; LoggerPtr log; @@ -1277,6 +1290,8 @@ ExternalLoader::ExternalLoader(const String & type_name_, LoggerPtr log_) : config_files_reader(std::make_unique(type_name_, log_)) , loading_dispatcher(std::make_unique( [this](auto && a, auto && b, auto && c) { return createObject(a, b, c); }, + [this](const IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) + { return updateObjectFromConfigWithoutReloading(const_cast(object), config, key_in_config); }, type_name_, log_)) , periodic_updater(std::make_unique(*config_files_reader, *loading_dispatcher)) diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index a5d83bdab50a..279157b6d515 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -213,6 +213,10 @@ class ExternalLoader protected: virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config, const String & repository_name) const = 0; + /// Updates the object from the configuration without reloading as much as possible. + virtual void updateObjectFromConfigWithoutReloading( + IExternalLoadable & /* object */, const Poco::Util::AbstractConfiguration & /* config */, const String & /* key_in_config */) const {} + private: void checkLoaded(const LoadResult & result, bool check_no_errors) const; void checkLoaded(const LoadResults & results, bool check_no_errors) const; diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index f716332dd24f..0c6f6a25b9dd 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -221,6 +221,10 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) bool move_to_ordinary = old_table_id.uuid != UUIDHelpers::Nil && new_table_id.uuid == UUIDHelpers::Nil; assert(old_table_id.uuid == new_table_id.uuid || move_to_atomic || move_to_ordinary); + /// It's better not to update an associated `IDictionary` directly here because it can be not loaded yet or + /// it can be in the process of loading or reloading right now. + /// The correct way is to update the dictionary's configuration first and then ask ExternalDictionariesLoader to reload our dictionary. + { std::lock_guard lock(dictionary_config_mutex); @@ -232,31 +236,28 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) configuration->remove("dictionary.uuid"); } + const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); + /// Dictionary is moving between databases of different engines or is renaming inside Ordinary database bool recreate_dictionary = old_table_id.uuid == UUIDHelpers::Nil || new_table_id.uuid == UUIDHelpers::Nil; if (recreate_dictionary) { - /// It's too hard to update both name and uuid, better to reload dictionary with new name + /// For an ordinary database the config repositories of dictionaries are identified by the full name (database name + dictionary name), + /// so we cannot change the dictionary name or the database name on the fly (without extra reloading) and have to recreate the config repository. removeDictionaryConfigurationFromRepository(); auto repository = std::make_unique(*this); - remove_repository_callback = getContext()->getExternalDictionariesLoader().addConfigRepository(std::move(repository)); - /// Dictionary will be reloaded lazily to avoid exceptions in the middle of renaming + remove_repository_callback = external_dictionaries_loader.addConfigRepository(std::move(repository)); + /// Dictionary will be now reloaded lazily. } else { - const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); - auto result = external_dictionaries_loader.getLoadResult(old_table_id.getInternalDictionaryName()); - - if (result.object) - { - const auto dictionary = std::static_pointer_cast(result.object); - dictionary->updateDictionaryName(new_table_id); - } - + /// For an atomic database dictionaries are identified inside the ExternalLoader by UUID, + /// so we can change the dictionary name or the database name on the fly (without extra reloading) because UUID doesn't change. external_dictionaries_loader.reloadConfig(old_table_id.getInternalDictionaryName()); - dictionary_name = new_table_id.getFullNameNotQuoted(); } + + dictionary_name = new_table_id.getFullNameNotQuoted(); } void StorageDictionary::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const From 4fe1c9a78f99b915dd4f31cbf37a2c3db6f49e1d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Mar 2024 00:07:53 +0100 Subject: [PATCH 3/9] Get rid of checking the "last modification time" when the configuration is generated to allow ExternalLoader::reloadConfig() work for generated configurations too. --- src/Interpreters/ExternalLoader.cpp | 4 ++-- .../ExternalLoaderDictionaryStorageConfigRepository.cpp | 5 ----- .../ExternalLoaderDictionaryStorageConfigRepository.h | 2 -- src/Interpreters/ExternalLoaderTempConfigRepository.cpp | 8 -------- src/Interpreters/ExternalLoaderTempConfigRepository.h | 2 -- src/Interpreters/ExternalLoaderXMLConfigRepository.cpp | 2 +- src/Interpreters/ExternalLoaderXMLConfigRepository.h | 2 +- src/Interpreters/IExternalLoaderConfigRepository.h | 2 +- src/Storages/StorageDictionary.cpp | 6 ------ src/Storages/StorageDictionary.h | 2 -- 10 files changed, 5 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index ec4290544754..3e162b1114f5 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -169,7 +169,7 @@ class ExternalLoader::LoadablesConfigReader : private boost::noncopyable private: struct FileInfo { - Poco::Timestamp last_update_time = 0; + std::optional last_update_time; bool in_use = true; // Whether the `FileInfo` should be destroyed because the correspondent file is deleted. Poco::AutoPtr file_contents; // Parsed contents of the file. std::unordered_map objects; @@ -268,7 +268,7 @@ class ExternalLoader::LoadablesConfigReader : private boost::noncopyable // is updated, but in the same second). // The solution to this is probably switching to std::filesystem // -- the work is underway to do so. - if (update_time_from_repository == file_info.last_update_time) + if (update_time_from_repository && (update_time_from_repository == file_info.last_update_time)) { file_info.in_use = true; return false; diff --git a/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.cpp b/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.cpp index 86f5a9ded0a4..042992fcd2ee 100644 --- a/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.cpp @@ -26,11 +26,6 @@ bool ExternalLoaderDictionaryStorageConfigRepository::exists(const std::string & return getName() == loadable_definition_name; } -Poco::Timestamp ExternalLoaderDictionaryStorageConfigRepository::getUpdateTime(const std::string &) -{ - return dictionary_storage.getUpdateTime(); -} - LoadablesConfigurationPtr ExternalLoaderDictionaryStorageConfigRepository::load(const std::string &) { return dictionary_storage.getConfiguration(); diff --git a/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.h b/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.h index 06d2b0faf750..f357df990e66 100644 --- a/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.h +++ b/src/Interpreters/ExternalLoaderDictionaryStorageConfigRepository.h @@ -19,8 +19,6 @@ class ExternalLoaderDictionaryStorageConfigRepository : public IExternalLoaderCo bool exists(const std::string & loadable_definition_name) override; - Poco::Timestamp getUpdateTime(const std::string & loadable_definition_name) override; - LoadablesConfigurationPtr load(const std::string & loadable_definition_name) override; private: diff --git a/src/Interpreters/ExternalLoaderTempConfigRepository.cpp b/src/Interpreters/ExternalLoaderTempConfigRepository.cpp index 10fc61a2ed0b..0a8fc1702948 100644 --- a/src/Interpreters/ExternalLoaderTempConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderTempConfigRepository.cpp @@ -28,14 +28,6 @@ bool ExternalLoaderTempConfigRepository::exists(const String & path_) } -Poco::Timestamp ExternalLoaderTempConfigRepository::getUpdateTime(const String & path_) -{ - if (!exists(path_)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Loadable {} not found", path_); - return creation_time; -} - - LoadablesConfigurationPtr ExternalLoaderTempConfigRepository::load(const String & path_) { if (!exists(path_)) diff --git a/src/Interpreters/ExternalLoaderTempConfigRepository.h b/src/Interpreters/ExternalLoaderTempConfigRepository.h index 92816dcb7f2f..a3c67b3eed60 100644 --- a/src/Interpreters/ExternalLoaderTempConfigRepository.h +++ b/src/Interpreters/ExternalLoaderTempConfigRepository.h @@ -18,14 +18,12 @@ class ExternalLoaderTempConfigRepository : public IExternalLoaderConfigRepositor std::set getAllLoadablesDefinitionNames() override; bool exists(const String & path) override; - Poco::Timestamp getUpdateTime(const String & path) override; LoadablesConfigurationPtr load(const String & path) override; private: String name; String path; LoadablesConfigurationPtr config; - Poco::Timestamp creation_time; }; } diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index 8b9584f1242b..a15f918f457b 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -24,7 +24,7 @@ ExternalLoaderXMLConfigRepository::ExternalLoaderXMLConfigRepository( { } -Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name) +std::optional ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name) { return FS::getModificationTimestamp(definition_entity_name); } diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.h b/src/Interpreters/ExternalLoaderXMLConfigRepository.h index 25646d72db3e..814817cad6f0 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.h +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.h @@ -26,7 +26,7 @@ class ExternalLoaderXMLConfigRepository : public IExternalLoaderConfigRepository bool exists(const std::string & definition_entity_name) override; /// Return xml-file modification time via stat call - Poco::Timestamp getUpdateTime(const std::string & definition_entity_name) override; + std::optional getUpdateTime(const std::string & definition_entity_name) override; /// May contain definition about several entities (several dictionaries in one .xml file) LoadablesConfigurationPtr load(const std::string & config_file) override; diff --git a/src/Interpreters/IExternalLoaderConfigRepository.h b/src/Interpreters/IExternalLoaderConfigRepository.h index 0d0c8acc01a3..6ba4add86e24 100644 --- a/src/Interpreters/IExternalLoaderConfigRepository.h +++ b/src/Interpreters/IExternalLoaderConfigRepository.h @@ -37,7 +37,7 @@ class IExternalLoaderConfigRepository virtual bool exists(const std::string & path) = 0; /// Returns entity last update time - virtual Poco::Timestamp getUpdateTime(const std::string & path) = 0; + virtual std::optional getUpdateTime(const std::string & /* path */) { return {}; } /// Load configuration from some concrete source to AbstractConfiguration virtual LoadablesConfigurationPtr load(const std::string & path) = 0; diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 0c6f6a25b9dd..ee76522fe48f 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -194,12 +194,6 @@ void StorageDictionary::removeDictionaryConfigurationFromRepository() remove_repository_callback.reset(); } -Poco::Timestamp StorageDictionary::getUpdateTime() const -{ - std::lock_guard lock(dictionary_config_mutex); - return update_time; -} - LoadablesConfigurationPtr StorageDictionary::getConfiguration() const { std::lock_guard lock(dictionary_config_mutex); diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 995a0192269c..5c6b2f6d8ad6 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -92,7 +92,6 @@ friend class TableFunctionDictionary; void alter(const AlterCommands & params, ContextPtr alter_context, AlterLockHolder &) override; - Poco::Timestamp getUpdateTime() const; LoadablesConfigurationPtr getConfiguration() const; String getDictionaryName() const { return dictionary_name; } @@ -102,7 +101,6 @@ friend class TableFunctionDictionary; const Location location; mutable std::mutex dictionary_config_mutex; - Poco::Timestamp update_time; LoadablesConfigurationPtr configuration; scope_guard remove_repository_callback; From 08d64f9e118329232a2149ff21e6ad47ffd37e1c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Mar 2024 00:42:12 +0100 Subject: [PATCH 4/9] Get rid of modifying a shared AbstractConfiguration (between StorageDictionary and ExternalDictionariesLoader). Always create a new configuration when we need to modify it. --- src/Common/Config/ConfigHelper.cpp | 29 +++++++++++++++++++++++++++++ src/Common/Config/ConfigHelper.h | 13 ++++++++----- src/Storages/StorageDictionary.cpp | 13 +++++++++---- src/Storages/StorageDictionary.h | 2 +- 4 files changed, 47 insertions(+), 10 deletions(-) diff --git a/src/Common/Config/ConfigHelper.cpp b/src/Common/Config/ConfigHelper.cpp index 69fe42de86c1..6de63fe78d73 100644 --- a/src/Common/Config/ConfigHelper.cpp +++ b/src/Common/Config/ConfigHelper.cpp @@ -1,5 +1,7 @@ #include #include +#include + namespace DB { @@ -7,6 +9,33 @@ namespace DB namespace ConfigHelper { +namespace +{ + void cloneImpl(Poco::Util::AbstractConfiguration & dest, const Poco::Util::AbstractConfiguration & src, const std::string & prefix = "") + { + std::vector keys; + src.keys(prefix, keys); + if (!keys.empty()) + { + std::string prefix_with_dot = prefix + "."; + for (const auto & key : keys) + cloneImpl(dest, src, prefix_with_dot + key); + } + else if (!prefix.empty()) + { + dest.setString(prefix, src.getRawString(prefix)); + } + } +} + + +Poco::AutoPtr clone(const Poco::Util::AbstractConfiguration & src) +{ + Poco::AutoPtr res(new Poco::Util::XMLConfiguration()); + cloneImpl(*res, src); + return res; +} + bool getBool(const Poco::Util::AbstractConfiguration & config, const std::string & key, bool default_, bool empty_as) { if (!config.has(key)) diff --git a/src/Common/Config/ConfigHelper.h b/src/Common/Config/ConfigHelper.h index a1b630fa7265..4f13bc4ad785 100644 --- a/src/Common/Config/ConfigHelper.h +++ b/src/Common/Config/ConfigHelper.h @@ -1,18 +1,21 @@ #pragma once +#include #include -namespace Poco + +namespace Poco::Util { - namespace Util - { - class AbstractConfiguration; - } + class AbstractConfiguration; } + namespace DB::ConfigHelper { +/// Clones a configuration. +Poco::AutoPtr clone(const Poco::Util::AbstractConfiguration & src); + /// The behavior is like `config.getBool(key, default_)`, /// except when the tag is empty (aka. self-closing), `empty_as` will be used instead of throwing Poco::Exception. bool getBool(const Poco::Util::AbstractConfiguration & config, const std::string & key, bool default_ = false, bool empty_as = true); diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index ee76522fe48f..414c66e9522b 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -221,13 +222,17 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) { std::lock_guard lock(dictionary_config_mutex); + auto new_configuration = ConfigHelper::clone(*configuration); + + new_configuration->setString("dictionary.database", new_table_id.database_name); + new_configuration->setString("dictionary.name", new_table_id.table_name); - configuration->setString("dictionary.database", new_table_id.database_name); - configuration->setString("dictionary.name", new_table_id.table_name); if (move_to_atomic) - configuration->setString("dictionary.uuid", toString(new_table_id.uuid)); + new_configuration->setString("dictionary.uuid", toString(new_table_id.uuid)); else if (move_to_ordinary) - configuration->remove("dictionary.uuid"); + new_configuration->remove("dictionary.uuid"); + + configuration = new_configuration; } const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 5c6b2f6d8ad6..a414e4586d9f 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -101,7 +101,7 @@ friend class TableFunctionDictionary; const Location location; mutable std::mutex dictionary_config_mutex; - LoadablesConfigurationPtr configuration; + LoadablesConfigurationPtr configuration TSA_GUARDED_BY(dictionary_config_mutex); scope_guard remove_repository_callback; From 5c60b1fab17823bd865394a54003336e5e0965a6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 15 Mar 2024 02:52:30 +0100 Subject: [PATCH 5/9] Remove unnecessary reloading while renaming a dictionary in Atomic database. --- .../AbstractConfigurationComparison.cpp | 91 ++++++++++++------- .../Config/AbstractConfigurationComparison.h | 6 ++ .../ExternalDictionariesLoader.cpp | 14 +++ src/Interpreters/ExternalDictionariesLoader.h | 3 + src/Interpreters/ExternalLoader.cpp | 18 +++- src/Interpreters/ExternalLoader.h | 4 + 6 files changed, 98 insertions(+), 38 deletions(-) diff --git a/src/Common/Config/AbstractConfigurationComparison.cpp b/src/Common/Config/AbstractConfigurationComparison.cpp index c00a1efeebe7..e983bd32f51e 100644 --- a/src/Common/Config/AbstractConfigurationComparison.cpp +++ b/src/Common/Config/AbstractConfigurationComparison.cpp @@ -19,6 +19,56 @@ namespace result += subkey; return result; } + + bool isSameConfigurationImpl(const Poco::Util::AbstractConfiguration & left, const String & left_key, + const Poco::Util::AbstractConfiguration & right, const String & right_key, + const std::unordered_set * ignore_keys) + { + if (&left == &right && left_key == right_key) + return true; + + /// Get the subkeys of the left and right configurations. + Poco::Util::AbstractConfiguration::Keys left_subkeys; + Poco::Util::AbstractConfiguration::Keys right_subkeys; + left.keys(left_key, left_subkeys); + right.keys(right_key, right_subkeys); + + if (ignore_keys) + { + std::erase_if(left_subkeys, [&](const String & key) { return ignore_keys->contains(key); }); + std::erase_if(right_subkeys, [&](const String & key) { return ignore_keys->contains(key); }); + } + + /// Check that the right configuration has the same set of subkeys as the left configuration. + if (left_subkeys.size() != right_subkeys.size()) + return false; + + if (left_subkeys.empty()) + { + if (left.hasProperty(left_key)) + { + return right.hasProperty(right_key) && (left.getRawString(left_key) == right.getRawString(right_key)); + } + else + { + return !right.hasProperty(right_key); + } + } + else + { + /// Go through all the subkeys and compare corresponding parts of the configurations. + std::unordered_set left_subkeys_set{left_subkeys.begin(), left_subkeys.end()}; + for (const auto & subkey : right_subkeys) + { + if (!left_subkeys_set.contains(subkey)) + return false; + + if (!isSameConfiguration(left, concatKeyAndSubKey(left_key, subkey), right, concatKeyAndSubKey(right_key, subkey))) + return false; + } + return true; + } + } } @@ -52,41 +102,14 @@ bool isSameConfigurationWithMultipleKeys(const Poco::Util::AbstractConfiguration bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key, const Poco::Util::AbstractConfiguration & right, const String & right_key) { - if (&left == &right && left_key == right_key) - return true; - - bool has_property = left.hasProperty(left_key); - if (has_property != right.hasProperty(right_key)) - return false; - if (has_property) - { - /// The left and right configurations contains values so we can compare them. - if (left.getRawString(left_key) != right.getRawString(right_key)) - return false; - } - - /// Get the subkeys of the left and right configurations. - Poco::Util::AbstractConfiguration::Keys subkeys; - left.keys(left_key, subkeys); - - { - /// Check that the right configuration has the same set of subkeys as the left configuration. - Poco::Util::AbstractConfiguration::Keys right_subkeys; - right.keys(right_key, right_subkeys); - std::unordered_set left_subkeys{subkeys.begin(), subkeys.end()}; - if ((left_subkeys.size() != right_subkeys.size()) || (left_subkeys.size() != subkeys.size())) - return false; - for (const auto & right_subkey : right_subkeys) - if (!left_subkeys.contains(right_subkey)) - return false; - } - - /// Go through all the subkeys and compare corresponding parts of the configurations. - for (const auto & subkey : subkeys) - if (!isSameConfiguration(left, concatKeyAndSubKey(left_key, subkey), right, concatKeyAndSubKey(right_key, subkey))) - return false; + return isSameConfigurationImpl(left, left_key, right, right_key, /* ignore_keys= */ nullptr); +} - return true; +bool isSameConfigurationIgnoringKeys(const Poco::Util::AbstractConfiguration & left, const String & left_key, + const Poco::Util::AbstractConfiguration & right, const String & right_key, + const std::unordered_set & ignore_keys) +{ + return isSameConfigurationImpl(left, left_key, right, right_key, !ignore_keys.empty() ? &ignore_keys : nullptr); } } diff --git a/src/Common/Config/AbstractConfigurationComparison.h b/src/Common/Config/AbstractConfigurationComparison.h index 75e0b77b1cac..edaff8d53637 100644 --- a/src/Common/Config/AbstractConfigurationComparison.h +++ b/src/Common/Config/AbstractConfigurationComparison.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace Poco::Util { @@ -33,6 +34,11 @@ namespace DB bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key, const Poco::Util::AbstractConfiguration & right, const String & right_key); + /// Returns true if specified subviews of the two configurations contains the same keys and values, but without checking specified keys. + bool isSameConfigurationIgnoringKeys(const Poco::Util::AbstractConfiguration & left, const String & left_key, + const Poco::Util::AbstractConfiguration & right, const String & right_key, + const std::unordered_set & ignore_keys); + inline bool operator==(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right) { return isSameConfiguration(left, right); diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 760f94d18c24..0591a6232d05 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include "config.h" @@ -41,6 +42,19 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( return DictionaryFactory::instance().create(name, config, key_in_config, getContext(), created_from_ddl); } +bool ExternalDictionariesLoader::doesConfigChangeRequiresReloadingObject(const Poco::Util::AbstractConfiguration & old_config, const String & old_key_in_config, + const Poco::Util::AbstractConfiguration & new_config, const String & new_key_in_config) const +{ + /// If the database is atomic then a dictionary can be renamed without reloading. + if (!old_config.getString(old_key_in_config + ".uuid", "").empty() && !new_config.getString(new_key_in_config + ".uuid", "").empty()) + { + static const std::unordered_set ignore_keys{"name", "database"}; + bool only_name_or_database_may_differ = isSameConfigurationIgnoringKeys(old_config, old_key_in_config, new_config, new_key_in_config, ignore_keys); + return !only_name_or_database_may_differ; + } + return true; +} + void ExternalDictionariesLoader::updateObjectFromConfigWithoutReloading(IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const { IDictionary & dict = static_cast(object); diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 3c226f998be2..f3aa8fa5f8ea 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -43,6 +43,9 @@ class ExternalDictionariesLoader : public ExternalLoader, WithContext LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config, const std::string & repository_name) const override; + bool doesConfigChangeRequiresReloadingObject(const Poco::Util::AbstractConfiguration & old_config, const String & old_key_in_config, + const Poco::Util::AbstractConfiguration & new_config, const String & new_key_in_config) const override; + void updateObjectFromConfigWithoutReloading( IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const override; diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 3e162b1114f5..6ab083f9fb78 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -88,6 +88,7 @@ namespace } }; + using DoesConfigChangeRequiresReloadingObjectFunction = std::function; using UpdateObjectFromConfigWithoutReloadingFunction = std::function; } @@ -402,10 +403,12 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable LoadingDispatcher( const CreateObjectFunction & create_object_function_, + const DoesConfigChangeRequiresReloadingObjectFunction & does_config_change_requires_reloading_object_, const UpdateObjectFromConfigWithoutReloadingFunction & update_object_from_config_without_reloading_, const String & type_name_, LoggerPtr log_) : create_object(create_object_function_) + , does_config_change_requires_reloading_object(does_config_change_requires_reloading_object_) , update_object_from_config_without_reloading(update_object_from_config_without_reloading_) , type_name(type_name_) , log(log_) @@ -472,10 +475,14 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable if (info.triedToLoad()) { - /// The object has been tried to load before, so it is currently in use or was in use - /// and we should try to reload it with the new config. - LOG_TRACE(log, "Will reload '{}' because its configuration has been changed and there were attempts to load it before", name); - startLoading(info, true); + bool config_change_requires_reloading = does_config_change_requires_reloading_object(*previous_config->config, previous_config->key_in_config, *new_config->config, new_config->key_in_config); + if (config_change_requires_reloading) + { + /// The object has been tried to load before, so it is currently in use or was in use + /// and we should try to reload it with the new config. + LOG_TRACE(log, "Will reload '{}' because its configuration has been changed and there were attempts to load it before", name); + startLoading(info, true); + } } } } @@ -1204,6 +1211,7 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable } const CreateObjectFunction create_object; + const DoesConfigChangeRequiresReloadingObjectFunction does_config_change_requires_reloading_object; const UpdateObjectFromConfigWithoutReloadingFunction update_object_from_config_without_reloading; const String type_name; LoggerPtr log; @@ -1290,6 +1298,8 @@ ExternalLoader::ExternalLoader(const String & type_name_, LoggerPtr log_) : config_files_reader(std::make_unique(type_name_, log_)) , loading_dispatcher(std::make_unique( [this](auto && a, auto && b, auto && c) { return createObject(a, b, c); }, + [this](const Poco::Util::AbstractConfiguration & config_1, const String & key_in_config_1, const Poco::Util::AbstractConfiguration & config_2, const String & key_in_config_2) + { return doesConfigChangeRequiresReloadingObject(config_1, key_in_config_1, config_2, key_in_config_2); }, [this](const IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) { return updateObjectFromConfigWithoutReloading(const_cast(object), config, key_in_config); }, type_name_, diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 279157b6d515..781d8ca9e7b3 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -213,6 +213,10 @@ class ExternalLoader protected: virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config, const String & repository_name) const = 0; + /// Returns whether the object must be reloaded after a specified change in its configuration. + virtual bool doesConfigChangeRequiresReloadingObject(const Poco::Util::AbstractConfiguration & /* old_config */, const String & /* old_key_in_config */, + const Poco::Util::AbstractConfiguration & /* new_config */, const String & /* new_key_in_config */) const { return true; /* always reload */ } + /// Updates the object from the configuration without reloading as much as possible. virtual void updateObjectFromConfigWithoutReloading( IExternalLoadable & /* object */, const Poco::Util::AbstractConfiguration & /* config */, const String & /* key_in_config */) const {} From 306f642d97ce51bed6fda59cac1c3e3465f6ef5a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 17 Mar 2024 11:57:01 +0100 Subject: [PATCH 6/9] Update dictionary's comment in the same safe manner too. --- src/Dictionaries/DictionaryFactory.cpp | 6 +----- src/Dictionaries/IDictionary.h | 14 ++++++++------ .../ExternalDictionariesLoader.cpp | 13 +++++++++---- src/Storages/StorageDictionary.cpp | 18 +++++++++--------- 4 files changed, 27 insertions(+), 24 deletions(-) diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index a566fb27de40..337121822eab 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -55,11 +55,7 @@ DictionaryPtr DictionaryFactory::create( if (found != registered_layouts.end()) { const auto & layout_creator = found->second.layout_create_function; - auto result = layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); - if (config.hasProperty(config_prefix + ".comment")) - result->setDictionaryComment(config.getString(config_prefix + ".comment")); - - return result; + return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); } } diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 9770e93669eb..bab80d3cd57c 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -332,12 +332,6 @@ class IDictionary : public IExternalLoadable, public IKeyValueEntity return std::static_pointer_cast(IExternalLoadable::shared_from_this()); } - void setDictionaryComment(String new_comment) - { - std::lock_guard lock{mutex}; - dictionary_comment = std::move(new_comment); - } - String getDictionaryComment() const { std::lock_guard lock{mutex}; @@ -454,6 +448,14 @@ class IDictionary : public IExternalLoadable, public IKeyValueEntity dictionary_id = new_dictionary_id; } + /// Internally called by ExternalDictionariesLoader. + /// In order to update the dictionary comment change its configuration first and then call ExternalDictionariesLoader::reloadConfig(). + void updateDictionaryComment(const String & new_dictionary_comment) + { + std::lock_guard lock{mutex}; + dictionary_comment = new_dictionary_comment; + } + private: mutable std::mutex mutex; StorageID dictionary_id TSA_GUARDED_BY(mutex); diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 0591a6232d05..68f22df29476 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -45,14 +45,17 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( bool ExternalDictionariesLoader::doesConfigChangeRequiresReloadingObject(const Poco::Util::AbstractConfiguration & old_config, const String & old_key_in_config, const Poco::Util::AbstractConfiguration & new_config, const String & new_key_in_config) const { + std::unordered_set ignore_keys; + ignore_keys.insert("comment"); /// We always can change the comment without reloading a dictionary. + /// If the database is atomic then a dictionary can be renamed without reloading. if (!old_config.getString(old_key_in_config + ".uuid", "").empty() && !new_config.getString(new_key_in_config + ".uuid", "").empty()) { - static const std::unordered_set ignore_keys{"name", "database"}; - bool only_name_or_database_may_differ = isSameConfigurationIgnoringKeys(old_config, old_key_in_config, new_config, new_key_in_config, ignore_keys); - return !only_name_or_database_may_differ; + ignore_keys.insert("name"); + ignore_keys.insert("database"); } - return true; + + return !isSameConfigurationIgnoringKeys(old_config, old_key_in_config, new_config, new_key_in_config, ignore_keys); } void ExternalDictionariesLoader::updateObjectFromConfigWithoutReloading(IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const @@ -67,6 +70,8 @@ void ExternalDictionariesLoader::updateObjectFromConfigWithoutReloading(IExterna if ((new_dictionary_id.uuid == old_dictionary_id.uuid) && (new_dictionary_id.uuid != UUIDHelpers::Nil)) dict.updateDictionaryID(new_dictionary_id); } + + dict.updateDictionaryComment(config.getString(key_in_config + ".comment", "")); } ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 414c66e9522b..a0c4156a7048 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -278,19 +278,19 @@ void StorageDictionary::alter(const AlterCommands & params, ContextPtr alter_con auto new_comment = getInMemoryMetadataPtr()->comment; - auto storage_id = getStorageID(); - const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); - auto result = external_dictionaries_loader.getLoadResult(storage_id.getInternalDictionaryName()); + /// It's better not to update an associated `IDictionary` directly here because it can be not loaded yet or + /// it can be in the process of loading or reloading right now. + /// The correct way is to update the dictionary's configuration first and then ask ExternalDictionariesLoader to reload our dictionary. - if (result.object) { - auto dictionary = std::static_pointer_cast(result.object); - auto * dictionary_non_const = const_cast(dictionary.get()); - dictionary_non_const->setDictionaryComment(new_comment); + std::lock_guard lock(dictionary_config_mutex); + auto new_configuration = ConfigHelper::clone(*configuration); + new_configuration->setString("dictionary.comment", new_comment); + configuration = new_configuration; } - std::lock_guard lock(dictionary_config_mutex); - configuration->setString("dictionary.comment", new_comment); + const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); + external_dictionaries_loader.reloadConfig(getStorageID().getInternalDictionaryName()); } void registerStorageDictionary(StorageFactory & factory) From c8375cd16777010a35d11340bf7b8c9c60c1fa1e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 18 Mar 2024 16:44:55 +0100 Subject: [PATCH 7/9] Better code: remove unnecessary usage of const_cast and std::function. --- src/Dictionaries/CacheDictionary.h | 2 +- src/Dictionaries/DirectDictionary.h | 2 +- src/Dictionaries/FlatDictionary.h | 2 +- src/Dictionaries/HashedArrayDictionary.h | 2 +- src/Dictionaries/HashedDictionary.h | 2 +- src/Dictionaries/IPAddressDictionary.h | 2 +- .../PolygonDictionaryImplementations.cpp | 6 +- .../PolygonDictionaryImplementations.h | 6 +- src/Dictionaries/RangeHashedDictionary.h | 2 +- src/Dictionaries/RegExpTreeDictionary.h | 2 +- ...alUserDefinedExecutableFunctionsLoader.cpp | 2 +- ...rnalUserDefinedExecutableFunctionsLoader.h | 2 +- .../UserDefinedExecutableFunction.h | 2 +- .../ExternalDictionariesLoader.cpp | 2 +- src/Interpreters/ExternalDictionariesLoader.h | 4 +- src/Interpreters/ExternalLoader.cpp | 60 +++++-------------- src/Interpreters/ExternalLoader.h | 7 ++- src/Interpreters/IExternalLoadable.h | 2 +- 18 files changed, 41 insertions(+), 68 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 8897fb40fa9e..50f59e3eddb9 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -98,7 +98,7 @@ class CacheDictionary final : public IDictionary bool supportUpdates() const override { return false; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared( getDictionaryID(), diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index 8bf183fd1ab0..d4b4cd8e698d 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -50,7 +50,7 @@ class DirectDictionary final : public IDictionary double getLoadFactor() const override { return 0; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone()); } diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index 231819ab4ee6..7b00ce574558 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -57,7 +57,7 @@ class FlatDictionary final : public IDictionary double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); } diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h index 4b2570ad928e..d4bb67b96066 100644 --- a/src/Dictionaries/HashedArrayDictionary.h +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -73,7 +73,7 @@ class HashedArrayDictionary final : public IDictionary double getLoadFactor() const override { return static_cast(total_element_count) / bucket_count; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); } diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 37812de29d98..4080c018289f 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -116,7 +116,7 @@ class HashedDictionary final : public IDictionary double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared>( getDictionaryID(), diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index cebdefcf7aee..bdd021570774 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -57,7 +57,7 @@ class IPAddressDictionary final : public IDictionary double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), configuration); } diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index 3feca2ec4104..64d294584303 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -29,7 +29,7 @@ PolygonDictionarySimple::PolygonDictionarySimple( { } -std::shared_ptr PolygonDictionarySimple::clone() const +std::shared_ptr PolygonDictionarySimple::clone() const { return std::make_shared( this->getDictionaryID(), @@ -76,7 +76,7 @@ PolygonDictionaryIndexEach::PolygonDictionaryIndexEach( } } -std::shared_ptr PolygonDictionaryIndexEach::clone() const +std::shared_ptr PolygonDictionaryIndexEach::clone() const { return std::make_shared( this->getDictionaryID(), @@ -126,7 +126,7 @@ PolygonDictionaryIndexCell::PolygonDictionaryIndexCell( { } -std::shared_ptr PolygonDictionaryIndexCell::clone() const +std::shared_ptr PolygonDictionaryIndexCell::clone() const { return std::make_shared( this->getDictionaryID(), diff --git a/src/Dictionaries/PolygonDictionaryImplementations.h b/src/Dictionaries/PolygonDictionaryImplementations.h index 912d501bcdea..690ff3a0f1bf 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.h +++ b/src/Dictionaries/PolygonDictionaryImplementations.h @@ -23,7 +23,7 @@ class PolygonDictionarySimple : public IPolygonDictionary DictionaryLifetime dict_lifetime_, Configuration configuration_); - std::shared_ptr clone() const override; + std::shared_ptr clone() const override; private: bool find(const Point & point, size_t & polygon_index) const override; @@ -47,7 +47,7 @@ class PolygonDictionaryIndexEach : public IPolygonDictionary int min_intersections_, int max_depth_); - std::shared_ptr clone() const override; + std::shared_ptr clone() const override; static constexpr size_t kMinIntersectionsDefault = 1; static constexpr size_t kMaxDepthDefault = 5; @@ -75,7 +75,7 @@ class PolygonDictionaryIndexCell : public IPolygonDictionary size_t min_intersections_, size_t max_depth_); - std::shared_ptr clone() const override; + std::shared_ptr clone() const override; static constexpr size_t kMinIntersectionsDefault = 1; static constexpr size_t kMaxDepthDefault = 5; diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 1a6ee7e81d4f..9f7dc5dc68a7 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -101,7 +101,7 @@ class RangeHashedDictionary final : public IDictionary double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { auto result = std::make_shared( getDictionaryID(), diff --git a/src/Dictionaries/RegExpTreeDictionary.h b/src/Dictionaries/RegExpTreeDictionary.h index 9e14abb49d00..a361c68879ef 100644 --- a/src/Dictionaries/RegExpTreeDictionary.h +++ b/src/Dictionaries/RegExpTreeDictionary.h @@ -86,7 +86,7 @@ class RegExpTreeDictionary : public IDictionary bool hasHierarchy() const override { return false; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared( getDictionaryID(), structure, source_ptr->clone(), configuration, use_vectorscan, flag_case_insensitive, flag_dotall); diff --git a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp index db98f88e53ba..a4f17aa1201b 100644 --- a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -120,7 +120,7 @@ void ExternalUserDefinedExecutableFunctionsLoader::reloadFunction(const std::str loadOrReload(user_defined_function_name); } -ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create(const std::string & name, +ExternalLoader::LoadableMutablePtr ExternalUserDefinedExecutableFunctionsLoader::createObject(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config, const std::string &) const diff --git a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.h b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.h index 1a62175eb0cc..eb86986c3911 100644 --- a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.h +++ b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.h @@ -28,7 +28,7 @@ class ExternalUserDefinedExecutableFunctionsLoader : public ExternalLoader, With void reloadFunction(const std::string & user_defined_function_name) const; protected: - LoadablePtr create(const std::string & name, + LoadableMutablePtr createObject(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config, const std::string & repository_name) const override; diff --git a/src/Functions/UserDefined/UserDefinedExecutableFunction.h b/src/Functions/UserDefined/UserDefinedExecutableFunction.h index 989f9dfe8959..d48be215c7dc 100644 --- a/src/Functions/UserDefined/UserDefinedExecutableFunction.h +++ b/src/Functions/UserDefined/UserDefinedExecutableFunction.h @@ -62,7 +62,7 @@ class UserDefinedExecutableFunction final : public IExternalLoadable return true; } - std::shared_ptr clone() const override + std::shared_ptr clone() const override { return std::make_shared(configuration, coordinator, lifetime); } diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 68f22df29476..34ad240d0895 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -32,7 +32,7 @@ ExternalDictionariesLoader::ExternalDictionariesLoader(ContextPtr global_context enablePeriodicUpdates(true); } -ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( +ExternalLoader::LoadableMutablePtr ExternalDictionariesLoader::createObject( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config, const std::string & repository_name) const { diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index f3aa8fa5f8ea..0fc58aa1edf7 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -40,8 +40,8 @@ class ExternalDictionariesLoader : public ExternalLoader, WithContext static void resetAll(); protected: - LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, - const std::string & key_in_config, const std::string & repository_name) const override; + LoadableMutablePtr createObject(const std::string & name, const Poco::Util::AbstractConfiguration & config, + const std::string & key_in_config, const std::string & repository_name) const override; bool doesConfigChangeRequiresReloadingObject(const Poco::Util::AbstractConfiguration & old_config, const String & old_key_in_config, const Poco::Util::AbstractConfiguration & new_config, const String & new_key_in_config) const override; diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 6ab083f9fb78..bd56a540128d 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -87,9 +87,6 @@ namespace lock = std::unique_lock(mutex); } }; - - using DoesConfigChangeRequiresReloadingObjectFunction = std::function; - using UpdateObjectFromConfigWithoutReloadingFunction = std::function; } @@ -98,10 +95,7 @@ namespace class ExternalLoader::LoadablesConfigReader : private boost::noncopyable { public: - LoadablesConfigReader(const String & type_name_, LoggerPtr log_) - : type_name(type_name_), log(log_) - { - } + LoadablesConfigReader(const String & type_name_, LoggerPtr log_) : type_name(type_name_), log(log_) { } ~LoadablesConfigReader() = default; using Repository = IExternalLoaderConfigRepository; @@ -397,21 +391,8 @@ class ExternalLoader::LoadablesConfigReader : private boost::noncopyable class ExternalLoader::LoadingDispatcher : private boost::noncopyable { public: - /// Called to load or reload an object. - using CreateObjectFunction = std::function; - - LoadingDispatcher( - const CreateObjectFunction & create_object_function_, - const DoesConfigChangeRequiresReloadingObjectFunction & does_config_change_requires_reloading_object_, - const UpdateObjectFromConfigWithoutReloadingFunction & update_object_from_config_without_reloading_, - const String & type_name_, - LoggerPtr log_) - : create_object(create_object_function_) - , does_config_change_requires_reloading_object(does_config_change_requires_reloading_object_) - , update_object_from_config_without_reloading(update_object_from_config_without_reloading_) - , type_name(type_name_) - , log(log_) + LoadingDispatcher(const String & type_name_, LoggerPtr log_, const ExternalLoader & external_loader_) + : type_name(type_name_), log(log_), external_loader(external_loader_) { } @@ -471,11 +452,11 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable if (config_changed) { if (info.object) - update_object_from_config_without_reloading(*info.object, *new_config->config, new_config->key_in_config); + external_loader.updateObjectFromConfigWithoutReloading(*info.object, *new_config->config, new_config->key_in_config); if (info.triedToLoad()) { - bool config_change_requires_reloading = does_config_change_requires_reloading_object(*previous_config->config, previous_config->key_in_config, *new_config->config, new_config->key_in_config); + bool config_change_requires_reloading = external_loader.doesConfigChangeRequiresReloadingObject(*previous_config->config, previous_config->key_in_config, *new_config->config, new_config->key_in_config); if (config_change_requires_reloading) { /// The object has been tried to load before, so it is currently in use or was in use @@ -786,7 +767,7 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable } String name; - LoadablePtr object; + LoadableMutablePtr object; std::shared_ptr config; TimePoint loading_start_time; TimePoint loading_end_time; @@ -1046,17 +1027,17 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable } /// Load one object, returns object ptr or exception. - std::pair + std::pair loadSingleObject(const String & name, const ObjectConfig & config, LoadablePtr previous_version) { /// Use `create_function` to perform the actual loading. /// It's much better to do it with `mutex` unlocked because the loading can take a lot of time /// and require access to other objects. - LoadablePtr new_object; + LoadableMutablePtr new_object; std::exception_ptr new_exception; try { - new_object = create_object(name, config, previous_version); + new_object = external_loader.createOrCloneObject(name, config, previous_version); } catch (...) { @@ -1070,7 +1051,7 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable const String & name, size_t loading_id, LoadablePtr previous_version, - LoadablePtr new_object, + LoadableMutablePtr new_object, std::exception_ptr new_exception, size_t error_count, const LoadingGuardForAsyncLoad &) @@ -1134,7 +1115,7 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable if (new_object) { - update_object_from_config_without_reloading(*new_object, *info->config->config, info->config->key_in_config); + external_loader.updateObjectFromConfigWithoutReloading(*new_object, *info->config->config, info->config->key_in_config); info->object = new_object; } @@ -1210,11 +1191,9 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable } } - const CreateObjectFunction create_object; - const DoesConfigChangeRequiresReloadingObjectFunction does_config_change_requires_reloading_object; - const UpdateObjectFromConfigWithoutReloadingFunction update_object_from_config_without_reloading; const String type_name; - LoggerPtr log; + const LoggerPtr log; + const ExternalLoader & external_loader; mutable std::mutex mutex; std::condition_variable event; @@ -1296,14 +1275,7 @@ class ExternalLoader::PeriodicUpdater : private boost::noncopyable ExternalLoader::ExternalLoader(const String & type_name_, LoggerPtr log_) : config_files_reader(std::make_unique(type_name_, log_)) - , loading_dispatcher(std::make_unique( - [this](auto && a, auto && b, auto && c) { return createObject(a, b, c); }, - [this](const Poco::Util::AbstractConfiguration & config_1, const String & key_in_config_1, const Poco::Util::AbstractConfiguration & config_2, const String & key_in_config_2) - { return doesConfigChangeRequiresReloadingObject(config_1, key_in_config_1, config_2, key_in_config_2); }, - [this](const IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) - { return updateObjectFromConfigWithoutReloading(const_cast(object), config, key_in_config); }, - type_name_, - log_)) + , loading_dispatcher(std::make_unique(type_name_, log_, *this)) , periodic_updater(std::make_unique(*config_files_reader, *loading_dispatcher)) , type_name(type_name_) , log(log_) @@ -1530,13 +1502,13 @@ void ExternalLoader::reloadConfig(const String & repository_name, const String & loading_dispatcher->setConfiguration(config_files_reader->read(repository_name, path)); } -ExternalLoader::LoadablePtr ExternalLoader::createObject( +ExternalLoader::LoadableMutablePtr ExternalLoader::createOrCloneObject( const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const { if (previous_version) return previous_version->clone(); - return create(name, *config.config, config.key_in_config, config.repository_name); + return createObject(name, *config.config, config.key_in_config, config.repository_name); } template ExternalLoader::LoadablePtr ExternalLoader::getLoadResult(const String &) const; diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 781d8ca9e7b3..c746ce39b2d0 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -50,6 +50,7 @@ class ExternalLoader { public: using LoadablePtr = std::shared_ptr; + using LoadableMutablePtr = std::shared_ptr; using Loadables = std::vector; using Status = ExternalLoaderStatus; @@ -211,7 +212,7 @@ class ExternalLoader void reloadConfig(const String & repository_name, const String & path) const; protected: - virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config, const String & repository_name) const = 0; + virtual LoadableMutablePtr createObject(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config, const String & repository_name) const = 0; /// Returns whether the object must be reloaded after a specified change in its configuration. virtual bool doesConfigChangeRequiresReloadingObject(const Poco::Util::AbstractConfiguration & /* old_config */, const String & /* old_key_in_config */, @@ -227,7 +228,7 @@ class ExternalLoader Strings getAllTriedToLoadNames() const; - LoadablePtr createObject(const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const; + LoadableMutablePtr createOrCloneObject(const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const; class LoadablesConfigReader; std::unique_ptr config_files_reader; @@ -239,7 +240,7 @@ class ExternalLoader std::unique_ptr periodic_updater; const String type_name; - LoggerPtr log; + const LoggerPtr log; }; } diff --git a/src/Interpreters/IExternalLoadable.h b/src/Interpreters/IExternalLoadable.h index 3c004508b0ad..1076bd88027c 100644 --- a/src/Interpreters/IExternalLoadable.h +++ b/src/Interpreters/IExternalLoadable.h @@ -43,7 +43,7 @@ class IExternalLoadable : public std::enable_shared_from_this /// If lifetime exceeded and isModified(), ExternalLoader replace current object with the result of clone(). virtual bool isModified() const = 0; /// Returns new object with the same configuration. Is used to update modified object when lifetime exceeded. - virtual std::shared_ptr clone() const = 0; + virtual std::shared_ptr clone() const = 0; }; } From 2f4b9b3a5b824ea255e1ea17c522f68d01cd0967 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 19 Mar 2024 16:08:20 +0100 Subject: [PATCH 8/9] Add few notes and a debug check. --- .../AbstractConfigurationComparison.cpp | 29 ++++++++++--------- .../Config/AbstractConfigurationComparison.h | 12 ++++---- src/Common/Config/ConfigHelper.h | 2 ++ .../ExternalDictionariesLoader.cpp | 2 +- 4 files changed, 24 insertions(+), 21 deletions(-) diff --git a/src/Common/Config/AbstractConfigurationComparison.cpp b/src/Common/Config/AbstractConfigurationComparison.cpp index e983bd32f51e..e241a540cc10 100644 --- a/src/Common/Config/AbstractConfigurationComparison.cpp +++ b/src/Common/Config/AbstractConfigurationComparison.cpp @@ -37,6 +37,12 @@ namespace { std::erase_if(left_subkeys, [&](const String & key) { return ignore_keys->contains(key); }); std::erase_if(right_subkeys, [&](const String & key) { return ignore_keys->contains(key); }); + +#if defined(ABORT_ON_LOGICAL_ERROR) + /// Compound `ignore_keys` are not yet implemented. + for (const auto & ignore_key : *ignore_keys) + chassert(ignore_key.find(".") == std::string_view::npos); +#endif } /// Check that the right configuration has the same set of subkeys as the left configuration. @@ -63,7 +69,7 @@ namespace if (!left_subkeys_set.contains(subkey)) return false; - if (!isSameConfiguration(left, concatKeyAndSubKey(left_key, subkey), right, concatKeyAndSubKey(right_key, subkey))) + if (!isSameConfigurationImpl(left, concatKeyAndSubKey(left_key, subkey), right, concatKeyAndSubKey(right_key, subkey), nullptr)) return false; } return true; @@ -82,6 +88,14 @@ bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const P return isSameConfiguration(left, key, right, key); } +bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key, + const Poco::Util::AbstractConfiguration & right, const String & right_key, + const std::unordered_set & ignore_keys) +{ + const auto * ignore_keys_ptr = !ignore_keys.empty() ? &ignore_keys : nullptr; + return isSameConfigurationImpl(left, left_key, right, right_key, ignore_keys_ptr); +} + bool isSameConfigurationWithMultipleKeys(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right, const String & root, const String & name) { if (&left == &right) @@ -99,17 +113,4 @@ bool isSameConfigurationWithMultipleKeys(const Poco::Util::AbstractConfiguration return true; } -bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key, - const Poco::Util::AbstractConfiguration & right, const String & right_key) -{ - return isSameConfigurationImpl(left, left_key, right, right_key, /* ignore_keys= */ nullptr); -} - -bool isSameConfigurationIgnoringKeys(const Poco::Util::AbstractConfiguration & left, const String & left_key, - const Poco::Util::AbstractConfiguration & right, const String & right_key, - const std::unordered_set & ignore_keys) -{ - return isSameConfigurationImpl(left, left_key, right, right_key, !ignore_keys.empty() ? &ignore_keys : nullptr); -} - } diff --git a/src/Common/Config/AbstractConfigurationComparison.h b/src/Common/Config/AbstractConfigurationComparison.h index edaff8d53637..e8f65a4afccd 100644 --- a/src/Common/Config/AbstractConfigurationComparison.h +++ b/src/Common/Config/AbstractConfigurationComparison.h @@ -11,6 +11,8 @@ namespace Poco::Util namespace DB { /// Returns true if two configurations contains the same keys and values. + /// NOTE: These functions assume no configuration has items having both children and a value + /// (i.e. items like "value"). bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right); @@ -31,13 +33,11 @@ namespace DB const String & key); /// Returns true if specified subviews of the two configurations contains the same keys and values. + /// If `ignore_keys` is specified then the function skips those keys while comparing + /// (even if their values differ, they're considered to be the same.) bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key, - const Poco::Util::AbstractConfiguration & right, const String & right_key); - - /// Returns true if specified subviews of the two configurations contains the same keys and values, but without checking specified keys. - bool isSameConfigurationIgnoringKeys(const Poco::Util::AbstractConfiguration & left, const String & left_key, - const Poco::Util::AbstractConfiguration & right, const String & right_key, - const std::unordered_set & ignore_keys); + const Poco::Util::AbstractConfiguration & right, const String & right_key, + const std::unordered_set & ignore_keys = {}); inline bool operator==(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right) { diff --git a/src/Common/Config/ConfigHelper.h b/src/Common/Config/ConfigHelper.h index 4f13bc4ad785..513438bd8593 100644 --- a/src/Common/Config/ConfigHelper.h +++ b/src/Common/Config/ConfigHelper.h @@ -14,6 +14,8 @@ namespace DB::ConfigHelper { /// Clones a configuration. +/// NOTE: This function assumes the source configuration doesn't have items having both children and a value +/// (i.e. items like "value"). Poco::AutoPtr clone(const Poco::Util::AbstractConfiguration & src); /// The behavior is like `config.getBool(key, default_)`, diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 34ad240d0895..f48ee61dab8e 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -55,7 +55,7 @@ bool ExternalDictionariesLoader::doesConfigChangeRequiresReloadingObject(const P ignore_keys.insert("database"); } - return !isSameConfigurationIgnoringKeys(old_config, old_key_in_config, new_config, new_key_in_config, ignore_keys); + return !isSameConfiguration(old_config, old_key_in_config, new_config, new_key_in_config, ignore_keys); } void ExternalDictionariesLoader::updateObjectFromConfigWithoutReloading(IExternalLoadable & object, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const From 29a52419972ca8bac81d56ced30d871495091cdb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 20 Mar 2024 17:18:08 +0100 Subject: [PATCH 9/9] fix build --- src/Common/Config/AbstractConfigurationComparison.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Config/AbstractConfigurationComparison.cpp b/src/Common/Config/AbstractConfigurationComparison.cpp index e241a540cc10..607b583cf314 100644 --- a/src/Common/Config/AbstractConfigurationComparison.cpp +++ b/src/Common/Config/AbstractConfigurationComparison.cpp @@ -41,7 +41,7 @@ namespace #if defined(ABORT_ON_LOGICAL_ERROR) /// Compound `ignore_keys` are not yet implemented. for (const auto & ignore_key : *ignore_keys) - chassert(ignore_key.find(".") == std::string_view::npos); + chassert(ignore_key.find('.') == std::string_view::npos); #endif }