ClickHouse/dbms/src/Common/Config/ConfigReloader.cpp

163 lines
4.6 KiB
C++
Raw Normal View History

#include "ConfigReloader.h"
2016-01-17 13:34:36 +00:00
#include <Poco/Util/Application.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <Common/setThreadName.h>
#include "ConfigProcessor.h"
2016-01-17 13:34:36 +00:00
namespace DB
{
2016-10-16 20:01:38 +00:00
constexpr decltype(ConfigReloader::reload_interval) ConfigReloader::reload_interval;
ConfigReloader::ConfigReloader(
const std::string & path_,
const std::string & include_from_path_,
zkutil::ZooKeeperNodeCache && zk_node_cache_,
Updater && updater_,
bool already_loaded)
: path(path_), include_from_path(include_from_path_)
, zk_node_cache(std::move(zk_node_cache_))
, updater(std::move(updater_))
2016-01-17 13:34:36 +00:00
{
if (!already_loaded)
reloadIfNewer(/* force = */ true, /* throw_on_error = */ true, /* fallback_to_preprocessed = */ true);
thread = std::thread(&ConfigReloader::run, this);
2016-01-17 13:34:36 +00:00
}
ConfigReloader::~ConfigReloader()
2016-01-17 13:34:36 +00:00
{
try
{
quit = true;
zk_node_cache.getChangedEvent().set();
thread.join();
}
catch (...)
{
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
}
2016-01-17 13:34:36 +00:00
}
void ConfigReloader::run()
2016-01-17 13:34:36 +00:00
{
setThreadName("ConfigReloader");
2016-01-17 13:34:36 +00:00
while (true)
{
try
{
bool zk_changed = zk_node_cache.getChangedEvent().tryWait(std::chrono::milliseconds(reload_interval).count());
if (quit)
return;
2016-10-23 10:52:32 +00:00
reloadIfNewer(zk_changed, /* throw_on_error = */ false, /* fallback_to_preprocessed = */ false);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
}
}
2016-01-17 13:34:36 +00:00
}
void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed)
2016-01-17 13:34:36 +00:00
{
FilesChangesTracker new_files = getNewFileList();
if (force || new_files.isDifferOrNewerThan(files))
{
ConfigProcessor config_processor(path);
ConfigProcessor::LoadedConfig loaded_config;
try
{
LOG_DEBUG(log, "Loading config `" << path << "'");
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (loaded_config.has_zk_includes)
loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
zk_node_cache, fallback_to_preprocessed);
}
catch (...)
{
if (throw_on_error)
throw;
tryLogCurrentException(log, "Error loading config from `" + path + "'");
return;
}
config_processor.savePreprocessedConfig(loaded_config);
/** We should remember last modification time if and only if config was sucessfully loaded
* Otherwise a race condition could occur during config files update:
* File is contain raw (and non-valid) data, therefore config is not applied.
* When file has been written (and contain valid data), we don't load new data since modification time remains the same.
*/
if (!loaded_config.loaded_from_preprocessed)
files = std::move(new_files);
try
{
updater(loaded_config.configuration);
}
catch (...)
{
if (throw_on_error)
throw;
tryLogCurrentException(log, "Error updating configuration from `" + path + "' config.");
}
}
}
struct ConfigReloader::FileWithTimestamp
{
std::string path;
time_t modification_time;
2016-01-17 13:34:36 +00:00
FileWithTimestamp(const std::string & path_, time_t modification_time_)
: path(path_), modification_time(modification_time_) {}
bool operator < (const FileWithTimestamp & rhs) const
{
return path < rhs.path;
}
2016-01-17 13:34:36 +00:00
static bool isTheSame(const FileWithTimestamp & lhs, const FileWithTimestamp & rhs)
{
return (lhs.modification_time == rhs.modification_time) && (lhs.path == rhs.path);
}
};
void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path)
{
if (!path.empty() && Poco::File(path).exists())
{
files.emplace(path, Poco::File(path).getLastModified().epochTime());
}
}
bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs)
{
return (files.size() != rhs.files.size()) ||
!std::equal(files.begin(), files.end(), rhs.files.begin(), FileWithTimestamp::isTheSame);
2016-01-17 13:34:36 +00:00
}
ConfigReloader::FilesChangesTracker ConfigReloader::getNewFileList() const
{
FilesChangesTracker file_list;
file_list.addIfExists(path);
file_list.addIfExists(include_from_path);
for (const auto & merge_path : ConfigProcessor::getConfigMergeFiles(path))
file_list.addIfExists(merge_path);
return file_list;
}
2016-01-17 13:34:36 +00:00
}