Merge pull request #7155 from ClickHouse/dictionaries_ddl

Preparation of ExternalLoader for Dictionaries DDL
This commit is contained in:
alesapin 2019-10-03 20:12:19 +03:00 committed by GitHub
commit cf094cd62b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 178 additions and 136 deletions

View File

@ -715,8 +715,8 @@ template <typename T>
PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds(const Attribute & attribute) const PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds(const Attribute & attribute) const
{ {
if (!sparse) if (!sparse)
return getIdsAttrImpl<T>(*std::get<CollectionPtrType<Key>>(attribute.maps)); return getIdsAttrImpl<T>(*std::get<CollectionPtrType<T>>(attribute.maps));
return getIdsAttrImpl<T>(*std::get<SparseCollectionPtrType<Key>>(attribute.sparse_maps)); return getIdsAttrImpl<T>(*std::get<SparseCollectionPtrType<T>>(attribute.sparse_maps));
} }
PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds() const PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds() const

View File

@ -29,6 +29,7 @@
#include <Interpreters/Quota.h> #include <Interpreters/Quota.h>
#include <Dictionaries/Embedded/GeoDictionariesLoader.h> #include <Dictionaries/Embedded/GeoDictionariesLoader.h>
#include <Interpreters/EmbeddedDictionaries.h> #include <Interpreters/EmbeddedDictionaries.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/ExternalDictionariesLoader.h> #include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalModelsLoader.h> #include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
@ -1320,8 +1321,8 @@ const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() cons
if (!this->global_context) if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = std::make_unique<ExternalLoaderConfigRepository>(); auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config, "dictionaries_config");
shared->external_dictionaries_loader.emplace(std::move(config_repository), config, *this->global_context); shared->external_dictionaries_loader.emplace(std::move(config_repository), *this->global_context);
} }
return *shared->external_dictionaries_loader; return *shared->external_dictionaries_loader;
} }
@ -1340,7 +1341,7 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const
if (!this->global_context) if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = std::make_unique<ExternalLoaderConfigRepository>(); auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(getConfigRef(), "models_config");
shared->external_models_loader.emplace(std::move(config_repository), *this->global_context); shared->external_models_loader.emplace(std::move(config_repository), *this->global_context);
} }
return *shared->external_models_loader; return *shared->external_models_loader;

View File

@ -7,15 +7,11 @@ namespace DB
/// Must not acquire Context lock in constructor to avoid possibility of deadlocks. /// Must not acquire Context lock in constructor to avoid possibility of deadlocks.
ExternalDictionariesLoader::ExternalDictionariesLoader( ExternalDictionariesLoader::ExternalDictionariesLoader(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository, ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
const Poco::Util::AbstractConfiguration & config, : ExternalLoader("external dictionary", &Logger::get("ExternalDictionariesLoader"))
Context & context_) , context(context_)
: ExternalLoader(config,
"external dictionary",
&Logger::get("ExternalDictionariesLoader")),
context(context_)
{ {
addConfigRepository(std::move(config_repository), {"dictionary", "name", "dictionaries_config"}); addConfigRepository(std::move(config_repository), {"dictionary", "name"});
enableAsyncLoading(true); enableAsyncLoading(true);
enablePeriodicUpdates(true); enablePeriodicUpdates(true);
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h> #include <Dictionaries/IDictionary.h>
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Interpreters/ExternalLoader.h> #include <Interpreters/ExternalLoader.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <memory> #include <memory>
@ -19,8 +20,7 @@ public:
/// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. /// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
ExternalDictionariesLoader( ExternalDictionariesLoader(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository, ExternalLoaderConfigRepositoryPtr config_repository,
const Poco::Util::AbstractConfiguration & config,
Context & context_); Context & context_);
DictPtr getDictionary(const std::string & name) const DictPtr getDictionary(const std::string & name) const

View File

@ -29,41 +29,39 @@ struct ExternalLoader::ObjectConfig
}; };
/** Reads configuration files and parses them as XML. /** Reads configurations from configuration repository and parses it.
* Stores parsed contents of the files along with their last modification time to
* avoid unnecessary parsing on repetetive reading.
*/ */
class ExternalLoader::ConfigFilesReader : private boost::noncopyable class ExternalLoader::LoadablesConfigReader : private boost::noncopyable
{ {
public: public:
ConfigFilesReader(const Poco::Util::AbstractConfiguration & main_config_, const String & type_name_, Logger * log_) LoadablesConfigReader(const String & type_name_, Logger * log_)
: main_config(main_config_), type_name(type_name_), log(log_) : type_name(type_name_), log(log_)
{ {
} }
~ConfigFilesReader() = default; ~LoadablesConfigReader() = default;
void addConfigRepository(std::unique_ptr<ExternalLoaderConfigRepository> repository, const ExternalLoaderConfigSettings & settings) void addConfigRepository(std::unique_ptr<IExternalLoaderConfigRepository> repository, const ExternalLoaderConfigSettings & settings)
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
repositories.emplace_back(std::move(repository), std::move(settings)); repositories.emplace_back(std::move(repository), std::move(settings));
} }
using ObjectConfigs = std::shared_ptr<const std::unordered_map<String /* object's name */, ObjectConfig>>; using ObjectConfigsPtr = std::shared_ptr<const std::unordered_map<String /* object's name */, ObjectConfig>>;
/// Reads configuration files. /// Reads configuration files.
ObjectConfigs read(bool ignore_last_modification_time = false) ObjectConfigsPtr read()
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
// Check last modification times of files and read those files which are new or changed. // Check last modification times of files and read those files which are new or changed.
if (!readFileInfos(ignore_last_modification_time)) if (!readLoadablesInfos())
return configs; // Nothing changed, so we can return the previous result. return configs; // Nothing changed, so we can return the previous result.
// Generate new result. // Generate new result.
auto new_configs = std::make_shared<std::unordered_map<String /* object's name */, ObjectConfig>>(); auto new_configs = std::make_shared<std::unordered_map<String /* object's name */, ObjectConfig>>();
for (const auto & [path, file_info] : file_infos) for (const auto & [path, loadable_info] : loadables_infos)
{ {
for (const auto & [name, config] : file_info.configs) for (const auto & [name, config] : loadable_info.configs)
{ {
auto already_added_it = new_configs->find(name); auto already_added_it = new_configs->find(name);
if (already_added_it != new_configs->end()) if (already_added_it != new_configs->end())
@ -84,42 +82,42 @@ public:
} }
private: private:
struct FileInfo struct LoadablesInfos
{ {
Poco::Timestamp last_modification_time; Poco::Timestamp last_update_time = 0;
std::vector<std::pair<String, ObjectConfig>> configs; // Parsed file's contents. std::vector<std::pair<String, ObjectConfig>> configs; // Parsed file's contents.
bool in_use = true; // Whether the `FileInfo` should be destroyed because the correspondent file is deleted. bool in_use = true; // Whether the ` LoadablesInfos` should be destroyed because the correspondent file is deleted.
}; };
/// Read files and store them to the map `file_infos`. /// Read files and store them to the map ` loadables_infos`.
bool readFileInfos(bool ignore_last_modification_time) bool readLoadablesInfos()
{ {
bool changed = false; bool changed = false;
for (auto & path_and_file_info : file_infos) for (auto & name_and_loadable_info : loadables_infos)
{ {
FileInfo & file_info = path_and_file_info.second; LoadablesInfos & loadable_info = name_and_loadable_info.second;
file_info.in_use = false; loadable_info.in_use = false;
} }
for (const auto & [repository, settings] : repositories) for (const auto & [repository, settings] : repositories)
{ {
const auto paths = repository->list(main_config, settings.path_setting_name); const auto names = repository->getAllLoadablesDefinitionNames();
for (const auto & path : paths) for (const auto & name : names)
{ {
auto it = file_infos.find(path); auto it = loadables_infos.find(name);
if (it != file_infos.end()) if (it != loadables_infos.end())
{ {
FileInfo & file_info = it->second; LoadablesInfos & loadable_info = it->second;
if (readFileInfo(*repository, path, settings, ignore_last_modification_time, file_info)) if (readLoadablesInfo(*repository, name, settings, loadable_info))
changed = true; changed = true;
} }
else else
{ {
FileInfo file_info; LoadablesInfos loadable_info;
if (readFileInfo(*repository, path, settings, true, file_info)) if (readLoadablesInfo(*repository, name, settings, loadable_info))
{ {
file_infos.emplace(path, std::move(file_info)); loadables_infos.emplace(name, std::move(loadable_info));
changed = true; changed = true;
} }
} }
@ -127,24 +125,23 @@ private:
} }
std::vector<String> deleted_files; std::vector<String> deleted_files;
for (auto & [path, file_info] : file_infos) for (auto & [path, loadable_info] : loadables_infos)
if (!file_info.in_use) if (!loadable_info.in_use)
deleted_files.emplace_back(path); deleted_files.emplace_back(path);
if (!deleted_files.empty()) if (!deleted_files.empty())
{ {
for (const String & deleted_file : deleted_files) for (const String & deleted_file : deleted_files)
file_infos.erase(deleted_file); loadables_infos.erase(deleted_file);
changed = true; changed = true;
} }
return changed; return changed;
} }
bool readFileInfo( bool readLoadablesInfo(
ExternalLoaderConfigRepository & repository, IExternalLoaderConfigRepository & repository,
const String & path, const String & path,
const ExternalLoaderConfigSettings & settings, const ExternalLoaderConfigSettings & settings,
bool ignore_last_modification_time, LoadablesInfos & loadable_info) const
FileInfo & file_info) const
{ {
try try
{ {
@ -154,14 +151,16 @@ private:
return false; return false;
} }
Poco::Timestamp last_modification_time = repository.getLastModificationTime(path); auto update_time_from_repository = repository.getUpdateTime(path);
if (!ignore_last_modification_time && (last_modification_time <= file_info.last_modification_time))
/// Actually it can't be less, but for sure we check less or equal
if (update_time_from_repository <= loadable_info.last_update_time)
{ {
file_info.in_use = true; loadable_info.in_use = true;
return false; return false;
} }
auto file_contents = repository.load(path, main_config.getString("path", DBMS_DEFAULT_PATH)); auto file_contents = repository.load(path);
/// get all objects' definitions /// get all objects' definitions
Poco::Util::AbstractConfiguration::Keys keys; Poco::Util::AbstractConfiguration::Keys keys;
@ -188,9 +187,9 @@ private:
configs_from_file.emplace_back(name, ObjectConfig{path, file_contents, key}); configs_from_file.emplace_back(name, ObjectConfig{path, file_contents, key});
} }
file_info.configs = std::move(configs_from_file); loadable_info.configs = std::move(configs_from_file);
file_info.last_modification_time = last_modification_time; loadable_info.last_update_time = update_time_from_repository;
file_info.in_use = true; loadable_info.in_use = true;
return true; return true;
} }
catch (...) catch (...)
@ -200,18 +199,17 @@ private:
} }
} }
const Poco::Util::AbstractConfiguration & main_config;
const String type_name; const String type_name;
Logger * log; Logger * log;
std::mutex mutex; std::mutex mutex;
std::vector<std::pair<std::unique_ptr<ExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories; std::vector<std::pair<std::unique_ptr<IExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories;
ObjectConfigs configs; ObjectConfigsPtr configs;
std::unordered_map<String /* config path */, FileInfo> file_infos; std::unordered_map<String /* config path */, LoadablesInfos> loadables_infos;
}; };
/** Manages loading and reloading objects. Uses configurations from the class ConfigFilesReader. /** Manages loading and reloading objects. Uses configurations from the class LoadablesConfigReader.
* Supports parallel loading. * Supports parallel loading.
*/ */
class ExternalLoader::LoadingDispatcher : private boost::noncopyable class ExternalLoader::LoadingDispatcher : private boost::noncopyable
@ -249,10 +247,10 @@ public:
} }
} }
using ObjectConfigs = ConfigFilesReader::ObjectConfigs; using ObjectConfigsPtr = LoadablesConfigReader::ObjectConfigsPtr;
/// Sets new configurations for all the objects. /// Sets new configurations for all the objects.
void setConfiguration(const ObjectConfigs & new_configs) void setConfiguration(const ObjectConfigsPtr & new_configs)
{ {
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
if (configs == new_configs) if (configs == new_configs)
@ -869,7 +867,7 @@ private:
mutable std::mutex mutex; mutable std::mutex mutex;
std::condition_variable event; std::condition_variable event;
ObjectConfigs configs; ObjectConfigsPtr configs;
std::unordered_map<String, Info> infos; std::unordered_map<String, Info> infos;
bool always_load_everything = false; bool always_load_everything = false;
bool enable_async_loading = false; bool enable_async_loading = false;
@ -884,7 +882,7 @@ class ExternalLoader::PeriodicUpdater : private boost::noncopyable
public: public:
static constexpr UInt64 check_period_sec = 5; static constexpr UInt64 check_period_sec = 5;
PeriodicUpdater(ConfigFilesReader & config_files_reader_, LoadingDispatcher & loading_dispatcher_) PeriodicUpdater(LoadablesConfigReader & config_files_reader_, LoadingDispatcher & loading_dispatcher_)
: config_files_reader(config_files_reader_), loading_dispatcher(loading_dispatcher_) : config_files_reader(config_files_reader_), loading_dispatcher(loading_dispatcher_)
{ {
} }
@ -934,7 +932,7 @@ private:
} }
} }
ConfigFilesReader & config_files_reader; LoadablesConfigReader & config_files_reader;
LoadingDispatcher & loading_dispatcher; LoadingDispatcher & loading_dispatcher;
mutable std::mutex mutex; mutable std::mutex mutex;
@ -944,8 +942,8 @@ private:
}; };
ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & main_config, const String & type_name_, Logger * log) ExternalLoader::ExternalLoader(const String & type_name_, Logger * log)
: config_files_reader(std::make_unique<ConfigFilesReader>(main_config, type_name_, log)) : config_files_reader(std::make_unique<LoadablesConfigReader>(type_name_, log))
, loading_dispatcher(std::make_unique<LoadingDispatcher>( , loading_dispatcher(std::make_unique<LoadingDispatcher>(
std::bind(&ExternalLoader::createObject, this, std::placeholders::_1, std::placeholders::_2, std::placeholders::_3, std::placeholders::_4), std::bind(&ExternalLoader::createObject, this, std::placeholders::_1, std::placeholders::_2, std::placeholders::_3, std::placeholders::_4),
type_name_, type_name_,
@ -958,7 +956,7 @@ ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & main_co
ExternalLoader::~ExternalLoader() = default; ExternalLoader::~ExternalLoader() = default;
void ExternalLoader::addConfigRepository( void ExternalLoader::addConfigRepository(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings) std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings)
{ {
config_files_reader->addConfigRepository(std::move(config_repository), config_settings); config_files_reader->addConfigRepository(std::move(config_repository), config_settings);
loading_dispatcher->setConfiguration(config_files_reader->read()); loading_dispatcher->setConfiguration(config_files_reader->read());

View File

@ -5,7 +5,7 @@
#include <unordered_map> #include <unordered_map>
#include <Core/Types.h> #include <Core/Types.h>
#include <Interpreters/IExternalLoadable.h> #include <Interpreters/IExternalLoadable.h>
#include <Interpreters/ExternalLoaderConfigRepository.h> #include <Interpreters/IExternalLoaderConfigRepository.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
@ -24,8 +24,6 @@ struct ExternalLoaderConfigSettings
{ {
std::string external_config; std::string external_config;
std::string external_name; std::string external_name;
std::string path_setting_name;
}; };
@ -78,12 +76,12 @@ public:
using LoadResults = std::vector<std::pair<String, LoadResult>>; using LoadResults = std::vector<std::pair<String, LoadResult>>;
ExternalLoader(const Poco::Util::AbstractConfiguration & main_config, const String & type_name_, Logger * log); ExternalLoader(const String & type_name_, Logger * log);
virtual ~ExternalLoader(); virtual ~ExternalLoader();
/// Adds a repository which will be used to read configurations from. /// Adds a repository which will be used to read configurations from.
void addConfigRepository( void addConfigRepository(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings); std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings);
/// Sets whether all the objects from the configuration should be always loaded (even those which are never used). /// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
void enableAlwaysLoadEverything(bool enable); void enableAlwaysLoadEverything(bool enable);
@ -157,8 +155,8 @@ private:
LoadablePtr createObject(const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const; LoadablePtr createObject(const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const;
class ConfigFilesReader; class LoadablesConfigReader;
std::unique_ptr<ConfigFilesReader> config_files_reader; std::unique_ptr<LoadablesConfigReader> config_files_reader;
class LoadingDispatcher; class LoadingDispatcher;
std::unique_ptr<LoadingDispatcher> loading_dispatcher; std::unique_ptr<LoadingDispatcher> loading_dispatcher;

View File

@ -1,31 +0,0 @@
#pragma once
#include <Poco/AutoPtr.h>
#include <Poco/Timestamp.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <set>
#include <string>
namespace DB
{
/** Config repository used by native server application.
* Represents files in local filesystem.
*/
class ExternalLoaderConfigRepository
{
public:
using Files = std::set<std::string>;
Files list(
const Poco::Util::AbstractConfiguration & config,
const std::string & path_key) const;
bool exists(const std::string & config_file) const;
Poco::Timestamp getLastModificationTime(const std::string & config_file) const;
Poco::AutoPtr<Poco::Util::AbstractConfiguration> load(const std::string & config_file, const std::string & preprocessed_dir = "") const;
};
}

View File

@ -1,4 +1,4 @@
#include <Interpreters/ExternalLoaderConfigRepository.h> #include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/Config/ConfigProcessor.h> #include <Common/Config/ConfigProcessor.h>
@ -12,13 +12,16 @@
namespace DB namespace DB
{ {
ExternalLoaderConfigRepository::Files ExternalLoaderConfigRepository::list( Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name)
const Poco::Util::AbstractConfiguration & config,
const std::string & path_key) const
{ {
Files files; return Poco::File(definition_entity_name).getLastModified();
}
auto patterns = getMultipleValuesFromConfig(config, "", path_key); std::set<std::string> ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames() const
{
std::set<std::string> files;
auto patterns = getMultipleValuesFromConfig(main_config, "", config_key);
for (auto & pattern : patterns) for (auto & pattern : patterns)
{ {
@ -27,7 +30,7 @@ ExternalLoaderConfigRepository::Files ExternalLoaderConfigRepository::list(
if (pattern[0] != '/') if (pattern[0] != '/')
{ {
const auto app_config_path = config.getString("config-file", "config.xml"); const auto app_config_path = main_config.getString("config-file", "config.xml");
const auto config_dir = Poco::Path{app_config_path}.parent().toString(); const auto config_dir = Poco::Path{app_config_path}.parent().toString();
const auto absolute_path = config_dir + pattern; const auto absolute_path = config_dir + pattern;
Poco::Glob::glob(absolute_path, files, 0); Poco::Glob::glob(absolute_path, files, 0);
@ -38,7 +41,7 @@ ExternalLoaderConfigRepository::Files ExternalLoaderConfigRepository::list(
Poco::Glob::glob(pattern, files, 0); Poco::Glob::glob(pattern, files, 0);
} }
for (Files::iterator it = files.begin(); it != files.end();) for (std::set<std::string>::iterator it = files.begin(); it != files.end();)
{ {
if (ConfigProcessor::isPreprocessedFile(*it)) if (ConfigProcessor::isPreprocessedFile(*it))
files.erase(it++); files.erase(it++);
@ -49,23 +52,17 @@ ExternalLoaderConfigRepository::Files ExternalLoaderConfigRepository::list(
return files; return files;
} }
bool ExternalLoaderConfigRepository::exists(const std::string & config_file) const bool ExternalLoaderXMLConfigRepository::exists(const std::string & definition_entity_name) const
{ {
return Poco::File(config_file).exists(); return Poco::File(definition_entity_name).exists();
} }
Poco::Timestamp ExternalLoaderConfigRepository::getLastModificationTime( Poco::AutoPtr<Poco::Util::AbstractConfiguration> ExternalLoaderXMLConfigRepository::load(
const std::string & config_file) const const std::string & config_file) const
{
return Poco::File(config_file).getLastModified();
}
Poco::AutoPtr<Poco::Util::AbstractConfiguration> ExternalLoaderConfigRepository::load(
const std::string & config_file, const std::string & preprocessed_dir) const
{ {
ConfigProcessor config_processor{config_file}; ConfigProcessor config_processor{config_file};
ConfigProcessor::LoadedConfig preprocessed = config_processor.loadConfig(); ConfigProcessor::LoadedConfig preprocessed = config_processor.loadConfig();
config_processor.savePreprocessedConfig(preprocessed, preprocessed_dir); config_processor.savePreprocessedConfig(preprocessed, main_config.getString("path", DBMS_DEFAULT_PATH));
return preprocessed.configuration; return preprocessed.configuration;
} }

View File

@ -0,0 +1,43 @@
#pragma once
#include <Core/Types.h>
#include <unordered_map>
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Poco/Timestamp.h>
namespace DB
{
/// XML config repository used by ExternalLoader.
/// Represents xml-files in local filesystem.
class ExternalLoaderXMLConfigRepository : public IExternalLoaderConfigRepository
{
public:
ExternalLoaderXMLConfigRepository(const Poco::Util::AbstractConfiguration & main_config_, const std::string & config_key_)
: main_config(main_config_)
, config_key(config_key_)
{
}
/// Return set of .xml files from path in main_config (config_key)
std::set<std::string> getAllLoadablesDefinitionNames() const override;
/// Checks that file with name exists on filesystem
bool exists(const std::string & definition_entity_name) const override;
/// Return xml-file modification time via stat call
Poco::Timestamp 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 & definition_entity_name) const override;
private:
/// Main server config (config.xml).
const Poco::Util::AbstractConfiguration & main_config;
/// Key which contains path to dicrectory with .xml configs for entries
std::string config_key;
};
}

View File

@ -11,14 +11,11 @@ namespace ErrorCodes
ExternalModelsLoader::ExternalModelsLoader( ExternalModelsLoader::ExternalModelsLoader(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository, ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
Context & context_) : ExternalLoader("external model", &Logger::get("ExternalModelsLoader"))
: ExternalLoader(context_.getConfigRef(), , context(context_)
"external model",
&Logger::get("ExternalModelsLoader")),
context(context_)
{ {
addConfigRepository(std::move(config_repository), {"model", "name", "models_config"}); addConfigRepository(std::move(config_repository), {"model", "name"});
enablePeriodicUpdates(true); enablePeriodicUpdates(true);
} }

View File

@ -19,7 +19,7 @@ public:
/// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. /// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
ExternalModelsLoader( ExternalModelsLoader(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository, ExternalLoaderConfigRepositoryPtr config_repository,
Context & context_); Context & context_);
ModelPtr getModel(const std::string & name) const ModelPtr getModel(const std::string & name) const

View File

@ -0,0 +1,43 @@
#pragma once
#include <Poco/AutoPtr.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Timestamp.h>
#include <memory>
#include <string>
#include <set>
namespace DB
{
using LoadablesConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/// Base interface for configurations source for Loadble objects, which can be
/// loaded with ExternalLoader. Configurations may came from filesystem (XML-files),
/// server memory (from database), etc. It's important that main result of this class
/// (LoadablesConfigurationPtr) may contain more than one loadable config,
/// each one with own key, which can be obtained with keys method,
/// for example multiple dictionaries can be defined in single .xml file.
class IExternalLoaderConfigRepository
{
public:
/// Return all available sources of loadables structure
/// (all .xml files from fs, all entities from database, etc)
virtual std::set<std::string> getAllLoadablesDefinitionNames() const = 0;
/// Checks that source of loadables configuration exist.
virtual bool exists(const std::string & loadable_definition_name) const = 0;
/// Returns entity last update time
virtual Poco::Timestamp getUpdateTime(const std::string & loadable_definition_name) = 0;
/// Load configuration from some concrete source to AbstractConfiguration
virtual LoadablesConfigurationPtr load(const std::string & loadable_definition_name) const = 0;
virtual ~IExternalLoaderConfigRepository() = default;
};
using ExternalLoaderConfigRepositoryPtr = std::unique_ptr<IExternalLoaderConfigRepository>;
}