Preparation for AST configurations for external dictionaries

This commit is contained in:
alesapin 2019-09-30 19:12:08 +03:00
parent bc6caba4db
commit 08681ac210
13 changed files with 232 additions and 184 deletions

View File

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

View File

@ -29,6 +29,7 @@
#include <Interpreters/Quota.h>
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
#include <Interpreters/EmbeddedDictionaries.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ExpressionActions.h>
@ -1320,8 +1321,8 @@ const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() cons
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = std::make_unique<ExternalLoaderConfigRepository>();
shared->external_dictionaries_loader.emplace(std::move(config_repository), config, *this->global_context);
auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config, "dictionaries_config");
shared->external_dictionaries_loader.emplace(std::move(config_repository), *this->global_context);
}
return *shared->external_dictionaries_loader;
}
@ -1340,7 +1341,7 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const
if (!this->global_context)
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);
}
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.
ExternalDictionariesLoader::ExternalDictionariesLoader(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository,
const Poco::Util::AbstractConfiguration & config,
Context & context_)
: ExternalLoader(config,
"external dictionary",
&Logger::get("ExternalDictionariesLoader")),
context(context_)
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
: ExternalLoader("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);
enablePeriodicUpdates(true);
}

View File

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

View File

@ -29,20 +29,18 @@ struct ExternalLoader::ObjectConfig
};
/** Reads configuration files and parses them as XML.
* Stores parsed contents of the files along with their last modification time to
* avoid unnecessary parsing on repetetive reading.
/** Reads configurations from configuration repository and parses it.
*/
class ExternalLoader::ConfigFilesReader : private boost::noncopyable
class ExternalLoader::LoadablesConfigReader : private boost::noncopyable
{
public:
ConfigFilesReader(const Poco::Util::AbstractConfiguration & main_config_, const String & type_name_, Logger * log_)
: main_config(main_config_), type_name(type_name_), log(log_)
LoadablesConfigReader(const String & type_name_, Logger * 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};
repositories.emplace_back(std::move(repository), std::move(settings));
@ -56,14 +54,14 @@ public:
std::lock_guard lock{mutex};
// Check last modification times of files and read those files which are new or changed.
if (!readFileInfos())
if (!readLoadablesInfos())
return configs; // Nothing changed, so we can return the previous result.
// Generate new result.
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);
if (already_added_it != new_configs->end())
@ -84,42 +82,41 @@ public:
}
private:
struct FileInfo
struct LoadablesInfos
{
Poco::Timestamp last_modification_time = 0;
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`.
bool readFileInfos()
/// Read files and store them to the map ` loadables_infos`.
bool readLoadablesInfos()
{
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;
file_info.in_use = false;
LoadablesInfos & loadable_info = name_and_loadable_info.second;
loadable_info.in_use = false;
}
for (const auto & [repository, settings] : repositories)
{
const auto paths = repository->list(main_config, settings.path_setting_name);
for (const auto & path : paths)
const auto names = repository->getAllLoadablesDefinitionNames();
for (const auto & name : names)
{
auto it = file_infos.find(path);
if (it != file_infos.end())
auto it = loadables_infos.find(name);
if (it != loadables_infos.end())
{
FileInfo & file_info = it->second;
if (readFileInfo(*repository, path, settings, file_info))
LoadablesInfos & loadable_info = it->second;
if (readLoadablesInfo(*repository, name, settings, loadable_info))
changed = true;
}
else
{
FileInfo file_info;
if (readFileInfo(*repository, path, settings, file_info))
LoadablesInfos loadable_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;
}
}
@ -127,23 +124,23 @@ private:
}
std::vector<String> deleted_files;
for (auto & [path, file_info] : file_infos)
if (!file_info.in_use)
for (auto & [path, loadable_info] : loadables_infos)
if (!loadable_info.in_use)
deleted_files.emplace_back(path);
if (!deleted_files.empty())
{
for (const String & deleted_file : deleted_files)
file_infos.erase(deleted_file);
loadables_infos.erase(deleted_file);
changed = true;
}
return changed;
}
bool readFileInfo(
ExternalLoaderConfigRepository & repository,
bool readLoadablesInfo(
IExternalLoaderConfigRepository & repository,
const String & path,
const ExternalLoaderConfigSettings & settings,
FileInfo & file_info) const
LoadablesInfos & loadable_info) const
{
try
{
@ -153,14 +150,13 @@ private:
return false;
}
Poco::Timestamp last_modification_time = repository.getLastModificationTime(path);
if (last_modification_time <= file_info.last_modification_time)
if (!repository.isUpdated(path))
{
file_info.in_use = true;
loadable_info.in_use = true;
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
Poco::Util::AbstractConfiguration::Keys keys;
@ -187,9 +183,8 @@ private:
configs_from_file.emplace_back(name, ObjectConfig{path, file_contents, key});
}
file_info.configs = std::move(configs_from_file);
file_info.last_modification_time = last_modification_time;
file_info.in_use = true;
loadable_info.configs = std::move(configs_from_file);
loadable_info.in_use = true;
return true;
}
catch (...)
@ -199,18 +194,17 @@ private:
}
}
const Poco::Util::AbstractConfiguration & main_config;
const String type_name;
Logger * log;
std::mutex mutex;
std::vector<std::pair<std::unique_ptr<ExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories;
std::vector<std::pair<std::unique_ptr<IExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories;
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.
*/
class ExternalLoader::LoadingDispatcher : private boost::noncopyable
@ -248,7 +242,7 @@ public:
}
}
using ObjectConfigsPtr = ConfigFilesReader::ObjectConfigsPtr;
using ObjectConfigsPtr = LoadablesConfigReader::ObjectConfigsPtr;
/// Sets new configurations for all the objects.
void setConfiguration(const ObjectConfigsPtr & new_configs)
@ -883,7 +877,7 @@ class ExternalLoader::PeriodicUpdater : private boost::noncopyable
public:
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_)
{
}
@ -933,7 +927,7 @@ private:
}
}
ConfigFilesReader & config_files_reader;
LoadablesConfigReader & config_files_reader;
LoadingDispatcher & loading_dispatcher;
mutable std::mutex mutex;
@ -943,8 +937,8 @@ private:
};
ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & main_config, const String & type_name_, Logger * log)
: config_files_reader(std::make_unique<ConfigFilesReader>(main_config, type_name_, log))
ExternalLoader::ExternalLoader(const String & type_name_, Logger * log)
: config_files_reader(std::make_unique<LoadablesConfigReader>(type_name_, log))
, loading_dispatcher(std::make_unique<LoadingDispatcher>(
std::bind(&ExternalLoader::createObject, this, std::placeholders::_1, std::placeholders::_2, std::placeholders::_3, std::placeholders::_4),
type_name_,
@ -957,7 +951,7 @@ ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & main_co
ExternalLoader::~ExternalLoader() = default;
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);
loading_dispatcher->setConfiguration(config_files_reader->read());

View File

@ -5,7 +5,7 @@
#include <unordered_map>
#include <Core/Types.h>
#include <Interpreters/IExternalLoadable.h>
#include <Interpreters/ExternalLoaderConfigRepository.h>
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <common/logger_useful.h>
@ -24,8 +24,6 @@ struct ExternalLoaderConfigSettings
{
std::string external_config;
std::string external_name;
std::string path_setting_name;
};
@ -78,12 +76,12 @@ public:
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();
/// Adds a repository which will be used to read configurations from.
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).
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;
class ConfigFilesReader;
std::unique_ptr<ConfigFilesReader> config_files_reader;
class LoadablesConfigReader;
std::unique_ptr<LoadablesConfigReader> config_files_reader;
class LoadingDispatcher;
std::unique_ptr<LoadingDispatcher> loading_dispatcher;

View File

@ -1,72 +0,0 @@
#include <Interpreters/ExternalLoaderConfigRepository.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Poco/Glob.h>
#include <Poco/File.h>
#include <Poco/Path.h>
namespace DB
{
ExternalLoaderConfigRepository::Files ExternalLoaderConfigRepository::list(
const Poco::Util::AbstractConfiguration & config,
const std::string & path_key) const
{
Files files;
auto patterns = getMultipleValuesFromConfig(config, "", path_key);
for (auto & pattern : patterns)
{
if (pattern.empty())
continue;
if (pattern[0] != '/')
{
const auto app_config_path = config.getString("config-file", "config.xml");
const auto config_dir = Poco::Path{app_config_path}.parent().toString();
const auto absolute_path = config_dir + pattern;
Poco::Glob::glob(absolute_path, files, 0);
if (!files.empty())
continue;
}
Poco::Glob::glob(pattern, files, 0);
}
for (Files::iterator it = files.begin(); it != files.end();)
{
if (ConfigProcessor::isPreprocessedFile(*it))
files.erase(it++);
else
++it;
}
return files;
}
bool ExternalLoaderConfigRepository::exists(const std::string & config_file) const
{
return Poco::File(config_file).exists();
}
Poco::Timestamp ExternalLoaderConfigRepository::getLastModificationTime(
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::LoadedConfig preprocessed = config_processor.loadConfig();
config_processor.savePreprocessedConfig(preprocessed, preprocessed_dir);
return preprocessed.configuration;
}
}

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

@ -0,0 +1,78 @@
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Poco/Glob.h>
#include <Poco/File.h>
#include <Poco/Path.h>
namespace DB
{
bool ExternalLoaderXMLConfigRepository::isUpdated(const std::string & definition_entity_name)
{
Poco::Timestamp last_modified = Poco::File(definition_entity_name).getLastModified();
auto itr = update_time_mapping.find(definition_entity_name);
if (itr == update_time_mapping.end() || last_modified > itr->second)
{
update_time_mapping[definition_entity_name] = last_modified;
return true;
}
return false;
}
std::set<std::string> ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames() const
{
std::set<std::string> files;
auto patterns = getMultipleValuesFromConfig(main_config, "", config_key);
for (auto & pattern : patterns)
{
if (pattern.empty())
continue;
if (pattern[0] != '/')
{
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 absolute_path = config_dir + pattern;
Poco::Glob::glob(absolute_path, files, 0);
if (!files.empty())
continue;
}
Poco::Glob::glob(pattern, files, 0);
}
for (std::set<std::string>::iterator it = files.begin(); it != files.end();)
{
if (ConfigProcessor::isPreprocessedFile(*it))
files.erase(it++);
else
++it;
}
return files;
}
bool ExternalLoaderXMLConfigRepository::exists(const std::string & definition_entity_name) const
{
return Poco::File(definition_entity_name).exists();
}
Poco::AutoPtr<Poco::Util::AbstractConfiguration> ExternalLoaderXMLConfigRepository::load(
const std::string & config_file) const
{
ConfigProcessor config_processor{config_file};
ConfigProcessor::LoadedConfig preprocessed = config_processor.loadConfig();
config_processor.savePreprocessedConfig(preprocessed, main_config.getString("path", DBMS_DEFAULT_PATH));
return preprocessed.configuration;
}
}

View File

@ -0,0 +1,45 @@
#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_)
{
}
std::set<std::string> getAllLoadablesDefinitionNames() const override;
bool exists(const std::string & definition_entity_name) const override;
bool isUpdated(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:
/// Simple map with last modification time with path -> timestamp,
/// modification time received by stat.
std::unordered_map<std::string, Poco::Timestamp> update_time_mapping;
/// 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(
std::unique_ptr<ExternalLoaderConfigRepository> config_repository,
Context & context_)
: ExternalLoader(context_.getConfigRef(),
"external model",
&Logger::get("ExternalModelsLoader")),
context(context_)
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
: ExternalLoader("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);
}

View File

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

View File

@ -0,0 +1,42 @@
#pragma once
#include <Poco/AutoPtr.h>
#include <Poco/Util/AbstractConfiguration.h>
#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;
/// Checks that entity was updated since last call of this method.
/// Assumes usage of some state and probably some mutex.
virtual bool isUpdated(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>;
}