diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b97b48d9c68..5a62bb88427 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -92,6 +92,7 @@ #include #include #include +#include #include #include @@ -663,7 +664,10 @@ try MainThreadStatus::getInstance(); - StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true)); + ServerSettings server_settings; + server_settings.loadSettingsFromConfig(config()); + + StackTrace::setShowAddresses(server_settings.show_addresses_in_stack_traces); #if USE_HDFS /// This will point libhdfs3 to the right location for its config. @@ -748,9 +752,9 @@ try // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will // ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well. GlobalThreadPool::initialize( - config().getUInt("max_thread_pool_size", 10000), - config().getUInt("max_thread_pool_free_size", 1000), - config().getUInt("thread_pool_queue_size", 10000)); + server_settings.max_thread_pool_size, + server_settings.max_thread_pool_free_size, + server_settings.thread_pool_queue_size); #if USE_AZURE_BLOB_STORAGE /// It makes sense to deinitialize libxml after joining of all threads @@ -766,9 +770,9 @@ try #endif IOThreadPool::initialize( - config().getUInt("max_io_thread_pool_size", 100), - config().getUInt("max_io_thread_pool_free_size", 0), - config().getUInt("io_thread_pool_queue_size", 10000)); + server_settings.max_io_thread_pool_size, + server_settings.max_io_thread_pool_free_size, + server_settings.io_thread_pool_queue_size); NamedCollectionUtils::loadFromConfig(config()); @@ -786,15 +790,15 @@ try } } - Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); + Poco::ThreadPool server_pool(3, server_settings.max_connections); std::mutex servers_lock; std::vector servers; std::vector servers_to_start_before_tables; /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, - config().getUInt("asynchronous_metrics_update_period_s", 1), - config().getUInt("asynchronous_heavy_metrics_update_period_s", 120), + server_settings.asynchronous_metrics_update_period_s, + server_settings.asynchronous_heavy_metrics_update_period_s, [&]() -> std::vector { std::vector metrics; @@ -809,7 +813,7 @@ try } ); - ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10)); + ConnectionCollector::init(global_context, server_settings.max_threads_for_connection_collector); bool has_zookeeper = config().has("zookeeper"); @@ -828,6 +832,9 @@ try Settings::checkNoSettingNamesAtTopLevel(config(), config_path); + /// We need to reload server settings because config could be updated via zookeeper. + server_settings.loadSettingsFromConfig(config()); + #if defined(OS_LINUX) std::string executable_path = getExecutablePath(); @@ -947,7 +954,7 @@ try std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); fs::path path = path_str; - std::string default_database = config().getString("default_database", "default"); + std::string default_database = server_settings.default_database.toString(); /// Check that the process user id matches the owner of the data. const auto effective_user_id = geteuid(); @@ -1038,21 +1045,18 @@ try LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); /// Storage with temporary data for processing of heavy queries. - if (auto temporary_policy = config().getString("tmp_policy", ""); !temporary_policy.empty()) + if (!server_settings.tmp_policy.value.empty()) { - size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - global_context->setTemporaryStoragePolicy(temporary_policy, max_size); + global_context->setTemporaryStoragePolicy(server_settings.tmp_policy, server_settings.max_temporary_data_on_disk_size); } - else if (auto temporary_cache = config().getString("temporary_data_in_cache", ""); !temporary_cache.empty()) + else if (!server_settings.temporary_data_in_cache.value.empty()) { - size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - global_context->setTemporaryStorageInCache(temporary_cache, max_size); + global_context->setTemporaryStorageInCache(server_settings.temporary_data_in_cache, server_settings.max_temporary_data_on_disk_size); } else { std::string temporary_path = config().getString("tmp_path", path / "tmp/"); - size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - global_context->setTemporaryStoragePath(temporary_path, max_size); + global_context->setTemporaryStoragePath(temporary_path, server_settings.max_temporary_data_on_disk_size); } /** Directory with 'flags': files indicating temporary settings for the server set by system administrator. @@ -1189,10 +1193,12 @@ try { Settings::checkNoSettingNamesAtTopLevel(*config, config_path); - /// Limit on total memory usage - size_t max_server_memory_usage = config->getUInt64("max_server_memory_usage", 0); + ServerSettings server_settings; + server_settings.loadSettingsFromConfig(*config); - double max_server_memory_usage_to_ram_ratio = config->getDouble("max_server_memory_usage_to_ram_ratio", 0.9); + size_t max_server_memory_usage = server_settings.max_server_memory_usage; + + double max_server_memory_usage_to_ram_ratio = server_settings.max_server_memory_usage_to_ram_ratio; size_t default_max_server_memory_usage = static_cast(memory_amount * max_server_memory_usage_to_ram_ratio); if (max_server_memory_usage == 0) @@ -1220,8 +1226,7 @@ try total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - bool allow_use_jemalloc_memory = config->getBool("allow_use_jemalloc_memory", true); - total_memory_tracker.setAllowUseJemallocMemory(allow_use_jemalloc_memory); + total_memory_tracker.setAllowUseJemallocMemory(server_settings.allow_use_jemalloc_memory); auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); @@ -1239,36 +1244,23 @@ try global_context->setRemoteHostFilter(*config); - /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) - if (config->has("max_table_size_to_drop")) - global_context->setMaxTableSizeToDrop(config->getUInt64("max_table_size_to_drop")); - - if (config->has("max_partition_size_to_drop")) - global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop")); + global_context->setMaxTableSizeToDrop(server_settings.max_table_size_to_drop); + global_context->setMaxPartitionSizeToDrop(server_settings.max_partition_size_to_drop); ConcurrencyControl::SlotCount concurrent_threads_soft_limit = ConcurrencyControl::Unlimited; - if (config->has("concurrent_threads_soft_limit_num")) + if (server_settings.concurrent_threads_soft_limit_num > 0 && server_settings.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit) + concurrent_threads_soft_limit = server_settings.concurrent_threads_soft_limit_num; + if (server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0) { - auto value = config->getUInt64("concurrent_threads_soft_limit_num", 0); - if (value > 0 && value < concurrent_threads_soft_limit) - concurrent_threads_soft_limit = value; - } - if (config->has("concurrent_threads_soft_limit_ratio_to_cores")) - { - auto value = config->getUInt64("concurrent_threads_soft_limit_ratio_to_cores", 0) * std::thread::hardware_concurrency(); + auto value = server_settings.concurrent_threads_soft_limit_ratio_to_cores * std::thread::hardware_concurrency(); if (value > 0 && value < concurrent_threads_soft_limit) concurrent_threads_soft_limit = value; } ConcurrencyControl::instance().setMaxConcurrency(concurrent_threads_soft_limit); - if (config->has("max_concurrent_queries")) - global_context->getProcessList().setMaxSize(config->getInt("max_concurrent_queries", 0)); - - if (config->has("max_concurrent_insert_queries")) - global_context->getProcessList().setMaxInsertQueriesAmount(config->getInt("max_concurrent_insert_queries", 0)); - - if (config->has("max_concurrent_select_queries")) - global_context->getProcessList().setMaxSelectQueriesAmount(config->getInt("max_concurrent_select_queries", 0)); + global_context->getProcessList().setMaxSize(server_settings.max_concurrent_queries); + global_context->getProcessList().setMaxInsertQueriesAmount(server_settings.max_concurrent_insert_queries); + global_context->getProcessList().setMaxSelectQueriesAmount(server_settings.max_concurrent_select_queries); if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); @@ -1277,56 +1269,43 @@ try /// Note: If you specified it in the top level config (not it config of default profile) /// then ClickHouse will use it exactly. /// This is done for backward compatibility. - if (global_context->areBackgroundExecutorsInitialized() && (config->has("background_pool_size") || config->has("background_merges_mutations_concurrency_ratio"))) + if (global_context->areBackgroundExecutorsInitialized() && (server_settings.background_pool_size.changed || server_settings.background_merges_mutations_concurrency_ratio.changed)) { - auto new_pool_size = config->getUInt64("background_pool_size", 16); - auto new_ratio = config->getUInt64("background_merges_mutations_concurrency_ratio", 2); + auto new_pool_size = server_settings.background_pool_size; + auto new_ratio = server_settings.background_merges_mutations_concurrency_ratio; global_context->getMergeMutateExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size * new_ratio); - auto new_scheduling_policy = config->getString("background_merges_mutations_scheduling_policy", "round_robin"); - global_context->getMergeMutateExecutor()->updateSchedulingPolicy(new_scheduling_policy); + global_context->getMergeMutateExecutor()->updateSchedulingPolicy(server_settings.background_merges_mutations_scheduling_policy.toString()); } - if (global_context->areBackgroundExecutorsInitialized() && config->has("background_move_pool_size")) + if (global_context->areBackgroundExecutorsInitialized() && server_settings.background_move_pool_size.changed) { - auto new_pool_size = config->getUInt64("background_move_pool_size"); + auto new_pool_size = server_settings.background_move_pool_size; global_context->getMovesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size); } - if (global_context->areBackgroundExecutorsInitialized() && config->has("background_fetches_pool_size")) + if (global_context->areBackgroundExecutorsInitialized() && server_settings.background_fetches_pool_size.changed) { - auto new_pool_size = config->getUInt64("background_fetches_pool_size"); + auto new_pool_size = server_settings.background_fetches_pool_size; global_context->getFetchesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size); } - if (global_context->areBackgroundExecutorsInitialized() && config->has("background_common_pool_size")) + if (global_context->areBackgroundExecutorsInitialized() && server_settings.background_common_pool_size.changed) { - auto new_pool_size = config->getUInt64("background_common_pool_size"); + auto new_pool_size = server_settings.background_common_pool_size; global_context->getCommonExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size); } - if (config->has("background_buffer_flush_schedule_pool_size")) - { - auto new_pool_size = config->getUInt64("background_buffer_flush_schedule_pool_size"); - global_context->getBufferFlushSchedulePool().increaseThreadsCount(new_pool_size); - } + if (server_settings.background_buffer_flush_schedule_pool_size.changed) + global_context->getBufferFlushSchedulePool().increaseThreadsCount(server_settings.background_buffer_flush_schedule_pool_size); - if (config->has("background_schedule_pool_size")) - { - auto new_pool_size = config->getUInt64("background_schedule_pool_size"); - global_context->getSchedulePool().increaseThreadsCount(new_pool_size); - } + if (server_settings.background_schedule_pool_size.changed) + global_context->getSchedulePool().increaseThreadsCount(server_settings.background_schedule_pool_size); - if (config->has("background_message_broker_schedule_pool_size")) - { - auto new_pool_size = config->getUInt64("background_message_broker_schedule_pool_size"); - global_context->getMessageBrokerSchedulePool().increaseThreadsCount(new_pool_size); - } + if (server_settings.background_message_broker_schedule_pool_size.changed) + global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings.background_message_broker_schedule_pool_size); - if (config->has("background_distributed_schedule_pool_size")) - { - auto new_pool_size = config->getUInt64("background_distributed_schedule_pool_size"); - global_context->getDistributedSchedulePool().increaseThreadsCount(new_pool_size); - } + if (server_settings.background_distributed_schedule_pool_size.changed) + global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings.background_distributed_schedule_pool_size); if (config->has("resources")) { @@ -1471,18 +1450,15 @@ try }); /// Limit on total number of concurrently executed queries. - global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0)); + global_context->getProcessList().setMaxSize(server_settings.max_concurrent_queries); /// Set up caches. - /// Lower cache size on low-memory systems. - double cache_size_to_ram_max_ratio = config().getDouble("cache_size_to_ram_max_ratio", 0.5); - size_t max_cache_size = static_cast(memory_amount * cache_size_to_ram_max_ratio); + size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); - /// Size of cache for uncompressed blocks. Zero means disabled. - String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", "SLRU"); + String uncompressed_cache_policy = server_settings.uncompressed_cache_policy; LOG_INFO(log, "Uncompressed cache policy name {}", uncompressed_cache_policy); - size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0); + size_t uncompressed_cache_size = server_settings.uncompressed_cache_size; if (uncompressed_cache_size > max_cache_size) { uncompressed_cache_size = max_cache_size; @@ -1504,9 +1480,8 @@ try global_context, settings.async_insert_threads)); - /// Size of cache for marks (index of MergeTree family of tables). - size_t mark_cache_size = config().getUInt64("mark_cache_size", 5368709120); - String mark_cache_policy = config().getString("mark_cache_policy", "SLRU"); + size_t mark_cache_size = server_settings.mark_cache_size; + String mark_cache_policy = server_settings.mark_cache_policy; if (!mark_cache_size) LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation."); if (mark_cache_size > max_cache_size) @@ -1517,20 +1492,14 @@ try } global_context->setMarkCache(mark_cache_size, mark_cache_policy); - /// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled. - size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0); - if (index_uncompressed_cache_size) - global_context->setIndexUncompressedCache(index_uncompressed_cache_size); + if (server_settings.index_uncompressed_cache_size) + global_context->setIndexUncompressedCache(server_settings.index_uncompressed_cache_size); - /// Size of cache for index marks (index of MergeTree skip indices). - size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", 0); - if (index_mark_cache_size) - global_context->setIndexMarkCache(index_mark_cache_size); + if (server_settings.index_mark_cache_size) + global_context->setIndexMarkCache(server_settings.index_mark_cache_size); - /// A cache for mmapped files. - size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. - if (mmap_cache_size) - global_context->setMMappedFileCache(mmap_cache_size); + if (server_settings.mmap_cache_size) + global_context->setMMappedFileCache(server_settings.mmap_cache_size); /// A cache for query results. global_context->setQueryCache(config()); @@ -1616,7 +1585,7 @@ try /// context is destroyed. /// In addition this object has to be created before the loading of the tables. std::unique_ptr dns_cache_updater; - if (config().has("disable_internal_dns_cache") && config().getInt("disable_internal_dns_cache")) + if (server_settings.disable_internal_dns_cache) { /// Disable DNS caching at all DNSResolver::instance().setDisableCacheFlag(); @@ -1626,7 +1595,7 @@ try { /// Initialize a watcher periodically updating DNS cache dns_cache_updater = std::make_unique( - global_context, config().getInt("dns_cache_update_period", 15), config().getUInt("dns_max_consecutive_failures", 5)); + global_context, server_settings.dns_cache_update_period, server_settings.dns_max_consecutive_failures); } if (dns_cache_updater) @@ -1890,7 +1859,7 @@ try LOG_INFO(log, "Closed all listening sockets."); /// Killing remaining queries. - if (!config().getBool("shutdown_wait_unfinished_queries", false)) + if (server_settings.shutdown_wait_unfinished_queries) global_context->getProcessList().killAllQueries(); if (current_connections) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp new file mode 100644 index 00000000000..c50a67b04c9 --- /dev/null +++ b/src/Core/ServerSettings.cpp @@ -0,0 +1,19 @@ +#include "ServerSettings.h" +#include + +namespace DB +{ + +IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) + +void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + for (auto setting : all()) + { + const auto & name = setting.getName(); + if (config.has(name)) + set(name, config.getString(name)); + } +} + +} diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h new file mode 100644 index 00000000000..d597cdf1c66 --- /dev/null +++ b/src/Core/ServerSettings.h @@ -0,0 +1,78 @@ +#pragma once + + +#include + + +namespace Poco::Util +{ +class AbstractConfiguration; +} + +namespace DB +{ + +#define SERVER_SETTINGS(M, ALIAS) \ + M(Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0) \ + M(Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0) \ + M(UInt64, max_thread_pool_size, 10000, "Max size for global thread pool.", 0) \ + M(UInt64, max_thread_pool_free_size, 1000, "Max free size for global thread pool.", 0) \ + M(UInt64, thread_pool_queue_size, 10000, "Queue size for global thread pool.", 0) \ + M(UInt64, max_io_thread_pool_size, 100, "Max size for IO thread pool.", 0) \ + M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ + M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ + M(Int32, max_connections, 1024, "Max server connections.", 0) \ + M(UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0) \ + M(UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating asynchronous metrics.", 0) \ + M(UInt32, max_threads_for_connection_collector, 100, "Max thread count for connection collector.", 0) \ + M(String, default_database, "default", "Default database name.", 0) \ + M(String, tmp_policy, "", "Policy for storage with temporary data.", 0) \ + M(UInt64, max_temporary_data_on_disk_size, 0, "Max data size for temporary storage.", 0) \ + M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \ + M(UInt64, max_server_memory_usage, 0, "Limit on total memory usage. Zero means Unlimited.", 0) \ + M(Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to ram ratio. Allows to lower max memory on low-memory systems.", 0) \ + M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \ + \ + M(UInt64, max_concurrent_queries, 0, "Limit on total number of concurrently executed queries. Zero means Unlimited.", 0) \ + M(UInt64, max_concurrent_insert_queries, 0, "Limit on total number of concurrently insert queries. Zero means Unlimited.", 0) \ + M(UInt64, max_concurrent_select_queries, 0, "Limit on total number of concurrently select queries. Zero means Unlimited.", 0) \ + \ + M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro ram max ratio. Allows to lower cache size on low-memory systems.", 0) \ + M(String, uncompressed_cache_policy, "SLRU", "Uncompressed cache policy name.", 0) \ + M(UInt64, uncompressed_cache_size, 0, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ + M(UInt64, mark_cache_size, 5368709120, "Size of cache for marks (index of MergeTree family of tables).", 0) \ + M(String, mark_cache_policy, "SLRU", "Mark cache policy name.", 0) \ + M(UInt64, index_uncompressed_cache_size, 0, "Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.", 0) \ + M(UInt64, index_mark_cache_size, 0, "Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.", 0) \ + M(UInt64, mmap_cache_size, 1000, "A cache for mmapped files.", 0) /* The choice of default is arbitrary. */ \ + \ + M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ + M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ + M(UInt32, dns_max_consecutive_failures, 1024, "Max server connections.", 0) \ + \ + M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ + M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ + M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means Unlimited.", 0) \ + M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 1024, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ + \ + M(UInt64, background_pool_size, 16, "Sets background pool size.", 0) \ + M(UInt64, background_merges_mutations_concurrency_ratio, 2, "Sets background merges mutations concurrency ratio.", 0) \ + M(String, background_merges_mutations_scheduling_policy, "round_robin", "Sets background merges mutations scheduling policy.", 0) \ + M(UInt64, background_move_pool_size, 8, "Sets background move pool size.", 0) \ + M(UInt64, background_fetches_pool_size, 8, "Sets background fetches pool size.", 0) \ + M(UInt64, background_common_pool_size, 8, "Sets background common pool size.", 0) \ + M(UInt64, background_buffer_flush_schedule_pool_size, 16, "Sets background flush schedule pool size.", 0) \ + M(UInt64, background_schedule_pool_size, 16, "Sets background schedule pool size.", 0) \ + M(UInt64, background_message_broker_schedule_pool_size, 16, "Sets background message broker schedule pool size.", 0) \ + M(UInt64, background_distributed_schedule_pool_size, 16, "Sets background distributed schedule pool size.", 0) \ + + +DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) + +struct ServerSettings : public BaseSettings +{ + void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config); +}; + +} + diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 4164bf1e27e..06cd53013ec 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -150,10 +150,16 @@ template struct SettingFieldNumber; template struct SettingFieldNumber; template struct SettingFieldNumber; template struct SettingFieldNumber; +template struct SettingFieldNumber; +template struct SettingFieldNumber; +template struct SettingFieldNumber; template struct SettingAutoWrapper>; template struct SettingAutoWrapper>; template struct SettingAutoWrapper>; +template struct SettingAutoWrapper>; +template struct SettingAutoWrapper>; +template struct SettingAutoWrapper>; namespace { diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index c6fe46c9f6b..3994e402c9a 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -55,7 +55,10 @@ struct SettingFieldNumber using SettingFieldUInt64 = SettingFieldNumber; using SettingFieldInt64 = SettingFieldNumber; +using SettingFieldUInt32 = SettingFieldNumber; +using SettingFieldInt32 = SettingFieldNumber; using SettingFieldFloat = SettingFieldNumber; +using SettingFieldDouble = SettingFieldNumber; using SettingFieldBool = SettingFieldNumber; /** Wraps any SettingField to support special value 'auto' that can be checked with `is_auto` flag. @@ -129,6 +132,9 @@ struct SettingAutoWrapper using SettingFieldUInt64Auto = SettingAutoWrapper; using SettingFieldInt64Auto = SettingAutoWrapper; using SettingFieldFloatAuto = SettingAutoWrapper; +using SettingFieldUInt32Auto = SettingAutoWrapper; +using SettingFieldInt32Auto = SettingAutoWrapper; +using SettingFieldDoubleAuto = SettingAutoWrapper; /* Similar to SettingFieldUInt64Auto with small differences to behave like regular UInt64, supported to compatibility. * When setting to 'auto' it becomes equal to the number of processor cores without taking into account SMT. diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp new file mode 100644 index 00000000000..0e803d720a9 --- /dev/null +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + {"value", std::make_shared()}, + {"changed", std::make_shared()}, + {"description", std::make_shared()}, + {"type", std::make_shared()}, + }; +} + +void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + const auto & config = context->getConfigRef(); + ServerSettings settings; + settings.loadSettingsFromConfig(config); + + for (const auto & setting : settings.all()) + { + const auto & setting_name = setting.getName(); + res_columns[0]->insert(setting_name); + res_columns[1]->insert(setting.getValueString()); + res_columns[2]->insert(setting.isValueChanged()); + res_columns[3]->insert(setting.getDescription()); + res_columns[4]->insert(setting.getTypeName()); + } +} + +} diff --git a/src/Storages/System/StorageSystemServerSettings.h b/src/Storages/System/StorageSystemServerSettings.h new file mode 100644 index 00000000000..b3aa8055853 --- /dev/null +++ b/src/Storages/System/StorageSystemServerSettings.h @@ -0,0 +1,27 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/** implements system table "settings", which allows to get information about the current settings. + */ +class StorageSystemServerSettings final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemServerSettings"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 07db151069f..61329ab834b 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -105,6 +106,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "functions"); attach(context, system_database, "events"); attach(context, system_database, "settings"); + attach(context, system_database, "server_settings"); attach(context, system_database, "settings_changes"); attach>(context, system_database, "merge_tree_settings"); attach>(context, system_database, "replicated_merge_tree_settings");