Skip to content

Commit

Permalink
Merge pull request #7155 from ClickHouse/dictionaries_ddl
Browse files Browse the repository at this point in the history
Preparation of ExternalLoader for Dictionaries DDL
  • Loading branch information
alesapin committed Oct 3, 2019
2 parents ddf171e + 8429f46 commit cf094cd
Show file tree
Hide file tree
Showing 12 changed files with 178 additions and 136 deletions.
4 changes: 2 additions & 2 deletions dbms/src/Dictionaries/HashedDictionary.cpp
Expand Up @@ -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
Expand Down
7 changes: 4 additions & 3 deletions dbms/src/Interpreters/Context.cpp
Expand Up @@ -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>
Expand Down Expand Up @@ -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;
}
Expand All @@ -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;
Expand Down
12 changes: 4 additions & 8 deletions dbms/src/Interpreters/ExternalDictionariesLoader.cpp
Expand Up @@ -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);
}
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Interpreters/ExternalDictionariesLoader.h
@@ -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>
Expand All @@ -19,8 +20,7 @@ class ExternalDictionariesLoader : public ExternalLoader

/// 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
Expand Down
112 changes: 55 additions & 57 deletions dbms/src/Interpreters/ExternalLoader.cpp
Expand Up @@ -29,41 +29,39 @@ 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));
}

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.
ObjectConfigs read(bool ignore_last_modification_time = false)
ObjectConfigsPtr read()
{
std::lock_guard lock{mutex};

// 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.

// 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())
Expand All @@ -84,67 +82,66 @@ class ExternalLoader::ConfigFilesReader : private boost::noncopyable
}

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.
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(bool ignore_last_modification_time)
/// 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, ignore_last_modification_time, 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, true, 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;
}
}
}
}

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,
bool ignore_last_modification_time,
FileInfo & file_info) const
LoadablesInfos & loadable_info) const
{
try
{
Expand All @@ -154,14 +151,16 @@ class ExternalLoader::ConfigFilesReader : private boost::noncopyable
return false;
}

Poco::Timestamp last_modification_time = repository.getLastModificationTime(path);
if (!ignore_last_modification_time && (last_modification_time <= file_info.last_modification_time))
auto update_time_from_repository = repository.getUpdateTime(path);

/// 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;
}

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;
Expand All @@ -188,9 +187,9 @@ class ExternalLoader::ConfigFilesReader : private boost::noncopyable
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.last_update_time = update_time_from_repository;
loadable_info.in_use = true;
return true;
}
catch (...)
Expand All @@ -200,18 +199,17 @@ class ExternalLoader::ConfigFilesReader : private boost::noncopyable
}
}

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;
ObjectConfigs configs;
std::unordered_map<String /* config path */, FileInfo> file_infos;
std::vector<std::pair<std::unique_ptr<IExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories;
ObjectConfigsPtr configs;
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
Expand Down Expand Up @@ -249,10 +247,10 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable
}
}

using ObjectConfigs = ConfigFilesReader::ObjectConfigs;
using ObjectConfigsPtr = LoadablesConfigReader::ObjectConfigsPtr;

/// Sets new configurations for all the objects.
void setConfiguration(const ObjectConfigs & new_configs)
void setConfiguration(const ObjectConfigsPtr & new_configs)
{
std::lock_guard lock{mutex};
if (configs == new_configs)
Expand Down Expand Up @@ -869,7 +867,7 @@ class ExternalLoader::LoadingDispatcher : private boost::noncopyable

mutable std::mutex mutex;
std::condition_variable event;
ObjectConfigs configs;
ObjectConfigsPtr configs;
std::unordered_map<String, Info> infos;
bool always_load_everything = false;
bool enable_async_loading = false;
Expand All @@ -884,7 +882,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_)
{
}
Expand Down Expand Up @@ -934,7 +932,7 @@ class ExternalLoader::PeriodicUpdater : private boost::noncopyable
}
}

ConfigFilesReader & config_files_reader;
LoadablesConfigReader & config_files_reader;
LoadingDispatcher & loading_dispatcher;

mutable std::mutex mutex;
Expand All @@ -944,8 +942,8 @@ class ExternalLoader::PeriodicUpdater : private boost::noncopyable
};


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_,
Expand All @@ -958,7 +956,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());
Expand Down
12 changes: 5 additions & 7 deletions dbms/src/Interpreters/ExternalLoader.h
Expand Up @@ -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>


Expand All @@ -24,8 +24,6 @@ struct ExternalLoaderConfigSettings
{
std::string external_config;
std::string external_name;

std::string path_setting_name;
};


Expand Down Expand Up @@ -78,12 +76,12 @@ class ExternalLoader

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);
Expand Down Expand Up @@ -157,8 +155,8 @@ class ExternalLoader

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;
Expand Down

0 comments on commit cf094cd

Please sign in to comment.