Fix server settings

This commit is contained in:
alesapin 2024-06-21 19:16:27 +02:00
parent 0dc01f4a94
commit 104dc8fa3e
7 changed files with 38 additions and 3 deletions

View File

@ -1528,6 +1528,8 @@ try
global_context->setMaxDictionaryNumToWarn(new_server_settings.max_dictionary_num_to_warn);
global_context->setMaxDatabaseNumToWarn(new_server_settings.max_database_num_to_warn);
global_context->setMaxPartNumToWarn(new_server_settings.max_part_num_to_warn);
/// Only for system.server_settings
global_context->setConfigReloaderInterval(new_server_settings.config_reload_interval_ms);
SlotCount concurrent_threads_soft_limit = UnlimitedSlots;
if (new_server_settings.concurrent_threads_soft_limit_num > 0 && new_server_settings.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)

View File

@ -27,6 +27,8 @@ class Context;
class ConfigReloader
{
public:
static constexpr auto DEFAULT_RELOAD_INTERVAL = std::chrono::milliseconds(2000);
using Updater = std::function<void(ConfigurationPtr, bool)>;
ConfigReloader(
@ -66,8 +68,6 @@ private:
FilesChangesTracker getNewFileList() const;
static constexpr auto DEFAULT_RELOAD_INTERVAL = std::chrono::milliseconds(2000);
LoggerPtr log = getLogger("ConfigReloader");
std::string config_path;

View File

@ -152,6 +152,7 @@ namespace DB
M(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \
M(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \
M(Double, gwp_asan_force_sample_probability, 0, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \
M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp

View File

@ -91,6 +91,7 @@
#include <Common/StackTrace.h>
#include <Common/Config/ConfigHelper.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/Config/ConfigReloader.h>
#include <Common/Config/AbstractConfigurationComparison.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ShellCommand.h>
@ -367,6 +368,9 @@ struct ContextSharedPart : boost::noncopyable
std::atomic_size_t max_view_num_to_warn = 10000lu;
std::atomic_size_t max_dictionary_num_to_warn = 1000lu;
std::atomic_size_t max_part_num_to_warn = 100000lu;
/// Only for system.server_settings, actually value stored in reloader itself
std::atomic_size_t config_reload_interval_ms = ConfigReloader::DEFAULT_RELOAD_INTERVAL.count();
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
String google_protos_path; /// Path to a directory that contains the proto files for the well-known Protobuf types.
mutable OnceFlag action_locks_manager_initialized;
@ -4500,6 +4504,16 @@ void Context::checkPartitionCanBeDropped(const String & database, const String &
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
}
void Context::setConfigReloaderInterval(size_t value_ms)
{
shared->config_reload_interval_ms.store(value_ms, std::memory_order_relaxed);
}
size_t Context::getConfigReloaderInterval() const
{
return shared->config_reload_interval_ms.load(std::memory_order_relaxed);
}
InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional<FormatSettings> & format_settings, std::optional<size_t> max_parsing_threads) const
{
return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, max_parsing_threads);

View File

@ -1161,6 +1161,9 @@ public:
size_t getMaxPartitionSizeToDrop() const;
void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const;
void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size, const size_t & max_partition_size_to_drop) const;
/// Only for system.server_settings, actual value is stored in ConfigReloader
void setConfigReloaderInterval(size_t value_ms);
size_t getConfigReloaderInterval() const;
/// Lets you select the compression codec according to the conditions described in the configuration file.
std::shared_ptr<ICompressionCodec> chooseCompressionCodec(size_t part_size, double part_size_ratio) const;

View File

@ -6,6 +6,7 @@
#include <IO/MMappedFileCache.h>
#include <IO/UncompressedCache.h>
#include <Interpreters/Context.h>
#include <Common/Config/ConfigReloader.h>
#include <Interpreters/ProcessList.h>
#include <Storages/MarkCache.h>
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
@ -84,7 +85,8 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context
{"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
{"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}},
{"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}}
{"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}},
{"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}}
};
if (context->areBackgroundExecutorsInitialized())

View File

@ -28,6 +28,12 @@ def test_reload_config(start_cluster):
f"Config reload interval set to 1000ms", look_behind_lines=2000
)
assert (
node.query(
"SELECT value from system.server_settings where name = 'config_reload_interval_ms'"
)
== "1000\n"
)
node.replace_in_config(
"/etc/clickhouse-server/config.d/config_reloader.xml",
"1000",
@ -37,3 +43,10 @@ def test_reload_config(start_cluster):
assert node.wait_for_log_line(
f"Config reload interval changed to 7777ms", look_behind_lines=2000
)
assert (
node.query(
"SELECT value from system.server_settings where name = 'config_reload_interval_ms'"
)
== "7777\n"
)