mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Add ability to change config reload interval
This commit is contained in:
parent
932e4bf982
commit
0dc01f4a94
@ -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.");
|
||||||
|
@ -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());
|
||||||
|
@ -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()
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
#!/usr/bin/env python3
|
@ -0,0 +1,4 @@
|
|||||||
|
<?xml version="1.0" encoding="utf-8"?>
|
||||||
|
<clickhouse>
|
||||||
|
<config_reload_interval_ms>1000</config_reload_interval_ms>
|
||||||
|
</clickhouse>
|
39
tests/integration/test_config_reloader_interval/test.py
Normal file
39
tests/integration/test_config_reloader_interval/test.py
Normal 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
|
||||||
|
)
|
Loading…
Reference in New Issue
Block a user