Add ability to change config reload interval

This commit is contained in:
alesapin 2024-06-21 18:24:20 +02:00
parent 932e4bf982
commit 0dc01f4a94
8 changed files with 78 additions and 20 deletions

View File

@ -577,8 +577,7 @@ try
#if USE_SSL #if USE_SSL
CertificateReloader::instance().tryLoad(*config); CertificateReloader::instance().tryLoad(*config);
#endif #endif
}, });
/* already_loaded = */ false); /// Reload it right now (initial loading)
SCOPE_EXIT({ SCOPE_EXIT({
LOG_INFO(log, "Shutting down."); LOG_INFO(log, "Shutting down.");

View File

@ -1690,8 +1690,7 @@ try
/// Must be the last. /// Must be the last.
latest_config = config; latest_config = config;
}, });
/* already_loaded = */ false); /// Reload it right now (initial loading)
const auto listen_hosts = getListenHosts(config()); const auto listen_hosts = getListenHosts(config());
const auto interserver_listen_hosts = getInterserverListenHosts(config()); const auto interserver_listen_hosts = getInterserverListenHosts(config());

View File

@ -880,8 +880,7 @@ void UsersConfigAccessStorage::load(
Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path); Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path);
parseFromConfig(*new_config); parseFromConfig(*new_config);
access_control.getChangesNotifier().sendNotifications(); access_control.getChangesNotifier().sendNotifications();
}, });
/* already_loaded = */ false);
} }
void UsersConfigAccessStorage::startPeriodicReloading() void UsersConfigAccessStorage::startPeriodicReloading()

View File

@ -19,8 +19,7 @@ ConfigReloader::ConfigReloader(
const std::string & preprocessed_dir_, const std::string & preprocessed_dir_,
zkutil::ZooKeeperNodeCache && zk_node_cache_, zkutil::ZooKeeperNodeCache && zk_node_cache_,
const zkutil::EventPtr & zk_changed_event_, const zkutil::EventPtr & zk_changed_event_,
Updater && updater_, Updater && updater_)
bool already_loaded)
: config_path(config_path_) : config_path(config_path_)
, extra_paths(extra_paths_) , extra_paths(extra_paths_)
, preprocessed_dir(preprocessed_dir_) , preprocessed_dir(preprocessed_dir_)
@ -28,10 +27,15 @@ ConfigReloader::ConfigReloader(
, zk_changed_event(zk_changed_event_) , zk_changed_event(zk_changed_event_)
, updater(std::move(updater_)) , updater(std::move(updater_))
{ {
if (!already_loaded) auto config = reloadIfNewer(/* force = */ true, /* throw_on_error = */ true, /* fallback_to_preprocessed = */ true, /* initial_loading = */ true);
reloadIfNewer(/* force = */ true, /* throw_on_error = */ true, /* fallback_to_preprocessed = */ true, /* initial_loading = */ true);
}
if (config.has_value())
reload_interval = std::chrono::milliseconds(config->configuration->getInt64("config_reload_interval_ms", DEFAULT_RELOAD_INTERVAL.count()));
else
reload_interval = DEFAULT_RELOAD_INTERVAL;
LOG_TRACE(log, "Config reload interval set to {}ms", reload_interval.count());
}
void ConfigReloader::start() void ConfigReloader::start()
{ {
@ -82,7 +86,17 @@ void ConfigReloader::run()
if (quit) if (quit)
return; return;
reloadIfNewer(zk_changed, /* throw_on_error = */ false, /* fallback_to_preprocessed = */ false, /* initial_loading = */ false); auto config = reloadIfNewer(zk_changed, /* throw_on_error = */ false, /* fallback_to_preprocessed = */ false, /* initial_loading = */ false);
if (config.has_value())
{
auto new_reload_interval = std::chrono::milliseconds(config->configuration->getInt64("config_reload_interval_ms", DEFAULT_RELOAD_INTERVAL.count()));
if (new_reload_interval != reload_interval)
{
reload_interval = new_reload_interval;
LOG_TRACE(log, "Config reload interval changed to {}ms", reload_interval.count());
}
}
} }
catch (...) catch (...)
{ {
@ -92,7 +106,7 @@ void ConfigReloader::run()
} }
} }
void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed, bool initial_loading) std::optional<ConfigProcessor::LoadedConfig> ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed, bool initial_loading)
{ {
std::lock_guard lock(reload_mutex); std::lock_guard lock(reload_mutex);
@ -120,7 +134,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
throw; throw;
tryLogCurrentException(log, "ZooKeeper error when loading config from '" + config_path + "'"); tryLogCurrentException(log, "ZooKeeper error when loading config from '" + config_path + "'");
return; return std::nullopt;
} }
catch (...) catch (...)
{ {
@ -128,7 +142,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
throw; throw;
tryLogCurrentException(log, "Error loading config from '" + config_path + "'"); tryLogCurrentException(log, "Error loading config from '" + config_path + "'");
return; return std::nullopt;
} }
config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir);
@ -154,11 +168,13 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
if (throw_on_error) if (throw_on_error)
throw; throw;
tryLogCurrentException(log, "Error updating configuration from '" + config_path + "' config."); tryLogCurrentException(log, "Error updating configuration from '" + config_path + "' config.");
return; return std::nullopt;
} }
LOG_DEBUG(log, "Loaded config '{}', performed update on configuration", config_path); LOG_DEBUG(log, "Loaded config '{}', performed update on configuration", config_path);
return loaded_config;
} }
return std::nullopt;
} }
struct ConfigReloader::FileWithTimestamp struct ConfigReloader::FileWithTimestamp

View File

@ -35,8 +35,7 @@ public:
const std::string & preprocessed_dir, const std::string & preprocessed_dir,
zkutil::ZooKeeperNodeCache && zk_node_cache, zkutil::ZooKeeperNodeCache && zk_node_cache,
const zkutil::EventPtr & zk_changed_event, const zkutil::EventPtr & zk_changed_event,
Updater && updater, Updater && updater);
bool already_loaded);
~ConfigReloader(); ~ConfigReloader();
@ -53,7 +52,7 @@ public:
private: private:
void run(); void run();
void reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed, bool initial_loading); std::optional<ConfigProcessor::LoadedConfig> reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed, bool initial_loading);
struct FileWithTimestamp; struct FileWithTimestamp;
@ -67,7 +66,7 @@ private:
FilesChangesTracker getNewFileList() const; FilesChangesTracker getNewFileList() const;
static constexpr auto reload_interval = std::chrono::seconds(2); static constexpr auto DEFAULT_RELOAD_INTERVAL = std::chrono::milliseconds(2000);
LoggerPtr log = getLogger("ConfigReloader"); LoggerPtr log = getLogger("ConfigReloader");
@ -85,6 +84,8 @@ private:
std::atomic<bool> quit{false}; std::atomic<bool> quit{false};
ThreadFromGlobalPool thread; ThreadFromGlobalPool thread;
std::chrono::milliseconds reload_interval = DEFAULT_RELOAD_INTERVAL;
/// Locked inside reloadIfNewer. /// Locked inside reloadIfNewer.
std::mutex reload_mutex; std::mutex reload_mutex;
}; };

View File

@ -0,0 +1 @@
#!/usr/bin/env python3

View File

@ -0,0 +1,4 @@
<?xml version="1.0" encoding="utf-8"?>
<clickhouse>
<config_reload_interval_ms>1000</config_reload_interval_ms>
</clickhouse>

View File

@ -0,0 +1,39 @@
#!/usr/bin/env python3
import pytest
import fnmatch
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=["configs/config_reloader.xml"],
)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_reload_config(start_cluster):
assert node.wait_for_log_line(
f"Config reload interval set to 1000ms", look_behind_lines=2000
)
node.replace_in_config(
"/etc/clickhouse-server/config.d/config_reloader.xml",
"1000",
"7777",
)
assert node.wait_for_log_line(
f"Config reload interval changed to 7777ms", look_behind_lines=2000
)