mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #7155 from ClickHouse/dictionaries_ddl
Preparation of ExternalLoader for Dictionaries DDL
This commit is contained in:
commit
cf094cd62b
@ -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
|
||||||
|
@ -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;
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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());
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
43
dbms/src/Interpreters/ExternalLoaderXMLConfigRepository.h
Normal file
43
dbms/src/Interpreters/ExternalLoaderXMLConfigRepository.h
Normal 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;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
43
dbms/src/Interpreters/IExternalLoaderConfigRepository.h
Normal file
43
dbms/src/Interpreters/IExternalLoaderConfigRepository.h
Normal 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>;
|
||||||
|
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user