mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Move ServerSettings to pImpl
This commit is contained in:
parent
786c2bc575
commit
99924ba2bc
@ -77,6 +77,42 @@ namespace Setting
|
||||
extern const SettingsLocalFSReadMethod storage_file_read_method;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsDouble cache_size_to_ram_max_ratio;
|
||||
extern const ServerSettingsUInt64 compiled_expression_cache_elements_size;
|
||||
extern const ServerSettingsUInt64 compiled_expression_cache_size;
|
||||
extern const ServerSettingsUInt64 database_catalog_drop_table_concurrency;
|
||||
extern const ServerSettingsString default_database;
|
||||
extern const ServerSettingsString index_mark_cache_policy;
|
||||
extern const ServerSettingsUInt64 index_mark_cache_size;
|
||||
extern const ServerSettingsDouble index_mark_cache_size_ratio;
|
||||
extern const ServerSettingsString index_uncompressed_cache_policy;
|
||||
extern const ServerSettingsUInt64 index_uncompressed_cache_size;
|
||||
extern const ServerSettingsDouble index_uncompressed_cache_size_ratio;
|
||||
extern const ServerSettingsUInt64 io_thread_pool_queue_size;
|
||||
extern const ServerSettingsString mark_cache_policy;
|
||||
extern const ServerSettingsUInt64 mark_cache_size;
|
||||
extern const ServerSettingsDouble mark_cache_size_ratio;
|
||||
extern const ServerSettingsUInt64 max_active_parts_loading_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_io_thread_pool_free_size;
|
||||
extern const ServerSettingsUInt64 max_io_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_outdated_parts_loading_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_parts_cleaning_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_server_memory_usage;
|
||||
extern const ServerSettingsDouble max_server_memory_usage_to_ram_ratio;
|
||||
extern const ServerSettingsUInt64 max_thread_pool_free_size;
|
||||
extern const ServerSettingsUInt64 max_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_unexpected_parts_loading_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 mmap_cache_size;
|
||||
extern const ServerSettingsBool show_addresses_in_stack_traces;
|
||||
extern const ServerSettingsUInt64 thread_pool_queue_size;
|
||||
extern const ServerSettingsString uncompressed_cache_policy;
|
||||
extern const ServerSettingsUInt64 uncompressed_cache_size;
|
||||
extern const ServerSettingsDouble uncompressed_cache_size_ratio;
|
||||
extern const ServerSettingsBool use_legacy_mongodb_integration;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -157,9 +193,9 @@ void LocalServer::initialize(Poco::Util::Application & self)
|
||||
server_settings.loadSettingsFromConfig(config());
|
||||
|
||||
GlobalThreadPool::initialize(
|
||||
server_settings.max_thread_pool_size,
|
||||
server_settings.max_thread_pool_free_size,
|
||||
server_settings.thread_pool_queue_size);
|
||||
server_settings[ServerSetting::max_thread_pool_size],
|
||||
server_settings[ServerSetting::max_thread_pool_free_size],
|
||||
server_settings[ServerSetting::thread_pool_queue_size]);
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
/// See the explanation near the same line in Server.cpp
|
||||
@ -170,17 +206,17 @@ void LocalServer::initialize(Poco::Util::Application & self)
|
||||
#endif
|
||||
|
||||
getIOThreadPool().initialize(
|
||||
server_settings.max_io_thread_pool_size,
|
||||
server_settings.max_io_thread_pool_free_size,
|
||||
server_settings.io_thread_pool_queue_size);
|
||||
server_settings[ServerSetting::max_io_thread_pool_size],
|
||||
server_settings[ServerSetting::max_io_thread_pool_free_size],
|
||||
server_settings[ServerSetting::io_thread_pool_queue_size]);
|
||||
|
||||
const size_t active_parts_loading_threads = server_settings.max_active_parts_loading_thread_pool_size;
|
||||
const size_t active_parts_loading_threads = server_settings[ServerSetting::max_active_parts_loading_thread_pool_size];
|
||||
getActivePartsLoadingThreadPool().initialize(
|
||||
active_parts_loading_threads,
|
||||
0, // We don't need any threads one all the parts will be loaded
|
||||
active_parts_loading_threads);
|
||||
|
||||
const size_t outdated_parts_loading_threads = server_settings.max_outdated_parts_loading_thread_pool_size;
|
||||
const size_t outdated_parts_loading_threads = server_settings[ServerSetting::max_outdated_parts_loading_thread_pool_size];
|
||||
getOutdatedPartsLoadingThreadPool().initialize(
|
||||
outdated_parts_loading_threads,
|
||||
0, // We don't need any threads one all the parts will be loaded
|
||||
@ -188,7 +224,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
|
||||
|
||||
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(active_parts_loading_threads);
|
||||
|
||||
const size_t unexpected_parts_loading_threads = server_settings.max_unexpected_parts_loading_thread_pool_size;
|
||||
const size_t unexpected_parts_loading_threads = server_settings[ServerSetting::max_unexpected_parts_loading_thread_pool_size];
|
||||
getUnexpectedPartsLoadingThreadPool().initialize(
|
||||
unexpected_parts_loading_threads,
|
||||
0, // We don't need any threads one all the parts will be loaded
|
||||
@ -196,16 +232,16 @@ void LocalServer::initialize(Poco::Util::Application & self)
|
||||
|
||||
getUnexpectedPartsLoadingThreadPool().setMaxTurboThreads(active_parts_loading_threads);
|
||||
|
||||
const size_t cleanup_threads = server_settings.max_parts_cleaning_thread_pool_size;
|
||||
const size_t cleanup_threads = server_settings[ServerSetting::max_parts_cleaning_thread_pool_size];
|
||||
getPartsCleaningThreadPool().initialize(
|
||||
cleanup_threads,
|
||||
0, // We don't need any threads one all the parts will be deleted
|
||||
cleanup_threads);
|
||||
|
||||
getDatabaseCatalogDropTablesThreadPool().initialize(
|
||||
server_settings.database_catalog_drop_table_concurrency,
|
||||
server_settings[ServerSetting::database_catalog_drop_table_concurrency],
|
||||
0, // We don't need any threads if there are no DROP queries.
|
||||
server_settings.database_catalog_drop_table_concurrency);
|
||||
server_settings[ServerSetting::database_catalog_drop_table_concurrency]);
|
||||
}
|
||||
|
||||
|
||||
@ -470,7 +506,7 @@ try
|
||||
UseSSL use_ssl;
|
||||
thread_status.emplace();
|
||||
|
||||
StackTrace::setShowAddresses(server_settings.show_addresses_in_stack_traces);
|
||||
StackTrace::setShowAddresses(server_settings[ServerSetting::show_addresses_in_stack_traces]);
|
||||
|
||||
setupSignalHandler();
|
||||
|
||||
@ -507,10 +543,10 @@ try
|
||||
/// Don't initialize DateLUT
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions(server_settings.use_legacy_mongodb_integration);
|
||||
registerTableFunctions(server_settings[ServerSetting::use_legacy_mongodb_integration]);
|
||||
registerDatabases();
|
||||
registerStorages(server_settings.use_legacy_mongodb_integration);
|
||||
registerDictionaries(server_settings.use_legacy_mongodb_integration);
|
||||
registerStorages(server_settings[ServerSetting::use_legacy_mongodb_integration]);
|
||||
registerDictionaries(server_settings[ServerSetting::use_legacy_mongodb_integration]);
|
||||
registerDisks(/* global_skip_access_check= */ true);
|
||||
registerFormats();
|
||||
|
||||
@ -659,8 +695,8 @@ void LocalServer::processConfig()
|
||||
|
||||
const size_t physical_server_memory = getMemoryAmount();
|
||||
|
||||
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 max_server_memory_usage = server_settings[ServerSetting::max_server_memory_usage];
|
||||
double max_server_memory_usage_to_ram_ratio = server_settings[ServerSetting::max_server_memory_usage_to_ram_ratio];
|
||||
|
||||
size_t default_max_server_memory_usage = static_cast<size_t>(physical_server_memory * max_server_memory_usage_to_ram_ratio);
|
||||
|
||||
@ -689,12 +725,12 @@ void LocalServer::processConfig()
|
||||
total_memory_tracker.setDescription("(total)");
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
|
||||
const double cache_size_to_ram_max_ratio = server_settings.cache_size_to_ram_max_ratio;
|
||||
const double cache_size_to_ram_max_ratio = server_settings[ServerSetting::cache_size_to_ram_max_ratio];
|
||||
const size_t max_cache_size = static_cast<size_t>(physical_server_memory * cache_size_to_ram_max_ratio);
|
||||
|
||||
String uncompressed_cache_policy = server_settings.uncompressed_cache_policy;
|
||||
size_t uncompressed_cache_size = server_settings.uncompressed_cache_size;
|
||||
double uncompressed_cache_size_ratio = server_settings.uncompressed_cache_size_ratio;
|
||||
String uncompressed_cache_policy = server_settings[ServerSetting::uncompressed_cache_policy];
|
||||
size_t uncompressed_cache_size = server_settings[ServerSetting::uncompressed_cache_size];
|
||||
double uncompressed_cache_size_ratio = server_settings[ServerSetting::uncompressed_cache_size_ratio];
|
||||
if (uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
uncompressed_cache_size = max_cache_size;
|
||||
@ -702,9 +738,9 @@ void LocalServer::processConfig()
|
||||
}
|
||||
global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size, uncompressed_cache_size_ratio);
|
||||
|
||||
String mark_cache_policy = server_settings.mark_cache_policy;
|
||||
size_t mark_cache_size = server_settings.mark_cache_size;
|
||||
double mark_cache_size_ratio = server_settings.mark_cache_size_ratio;
|
||||
String mark_cache_policy = server_settings[ServerSetting::mark_cache_policy];
|
||||
size_t mark_cache_size = server_settings[ServerSetting::mark_cache_size];
|
||||
double mark_cache_size_ratio = server_settings[ServerSetting::mark_cache_size_ratio];
|
||||
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)
|
||||
@ -714,9 +750,9 @@ void LocalServer::processConfig()
|
||||
}
|
||||
global_context->setMarkCache(mark_cache_policy, mark_cache_size, mark_cache_size_ratio);
|
||||
|
||||
String index_uncompressed_cache_policy = server_settings.index_uncompressed_cache_policy;
|
||||
size_t index_uncompressed_cache_size = server_settings.index_uncompressed_cache_size;
|
||||
double index_uncompressed_cache_size_ratio = server_settings.index_uncompressed_cache_size_ratio;
|
||||
String index_uncompressed_cache_policy = server_settings[ServerSetting::index_uncompressed_cache_policy];
|
||||
size_t index_uncompressed_cache_size = server_settings[ServerSetting::index_uncompressed_cache_size];
|
||||
double index_uncompressed_cache_size_ratio = server_settings[ServerSetting::index_uncompressed_cache_size_ratio];
|
||||
if (index_uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
index_uncompressed_cache_size = max_cache_size;
|
||||
@ -724,9 +760,9 @@ void LocalServer::processConfig()
|
||||
}
|
||||
global_context->setIndexUncompressedCache(index_uncompressed_cache_policy, index_uncompressed_cache_size, index_uncompressed_cache_size_ratio);
|
||||
|
||||
String index_mark_cache_policy = server_settings.index_mark_cache_policy;
|
||||
size_t index_mark_cache_size = server_settings.index_mark_cache_size;
|
||||
double index_mark_cache_size_ratio = server_settings.index_mark_cache_size_ratio;
|
||||
String index_mark_cache_policy = server_settings[ServerSetting::index_mark_cache_policy];
|
||||
size_t index_mark_cache_size = server_settings[ServerSetting::index_mark_cache_size];
|
||||
double index_mark_cache_size_ratio = server_settings[ServerSetting::index_mark_cache_size_ratio];
|
||||
if (index_mark_cache_size > max_cache_size)
|
||||
{
|
||||
index_mark_cache_size = max_cache_size;
|
||||
@ -734,7 +770,7 @@ void LocalServer::processConfig()
|
||||
}
|
||||
global_context->setIndexMarkCache(index_mark_cache_policy, index_mark_cache_size, index_mark_cache_size_ratio);
|
||||
|
||||
size_t mmap_cache_size = server_settings.mmap_cache_size;
|
||||
size_t mmap_cache_size = server_settings[ServerSetting::mmap_cache_size];
|
||||
if (mmap_cache_size > max_cache_size)
|
||||
{
|
||||
mmap_cache_size = max_cache_size;
|
||||
@ -746,8 +782,8 @@ void LocalServer::processConfig()
|
||||
global_context->setQueryCache(0, 0, 0, 0);
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
size_t compiled_expression_cache_max_size_in_bytes = server_settings.compiled_expression_cache_size;
|
||||
size_t compiled_expression_cache_max_elements = server_settings.compiled_expression_cache_elements_size;
|
||||
size_t compiled_expression_cache_max_size_in_bytes = server_settings[ServerSetting::compiled_expression_cache_size];
|
||||
size_t compiled_expression_cache_max_elements = server_settings[ServerSetting::compiled_expression_cache_elements_size];
|
||||
CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements);
|
||||
#endif
|
||||
|
||||
@ -767,7 +803,7 @@ void LocalServer::processConfig()
|
||||
/// We load temporary database first, because projections need it.
|
||||
DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase();
|
||||
|
||||
std::string default_database = server_settings.default_database;
|
||||
std::string default_database = server_settings[ServerSetting::default_database];
|
||||
DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context));
|
||||
global_context->setCurrentDatabase(default_database);
|
||||
|
||||
|
@ -164,6 +164,123 @@ namespace MergeTreeSetting
|
||||
extern const MergeTreeSettingsBool allow_remote_fs_zero_copy_replication;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt32 asynchronous_heavy_metrics_update_period_s;
|
||||
extern const ServerSettingsUInt32 asynchronous_metrics_update_period_s;
|
||||
extern const ServerSettingsBool async_insert_queue_flush_on_shutdown;
|
||||
extern const ServerSettingsUInt64 async_insert_threads;
|
||||
extern const ServerSettingsBool async_load_databases;
|
||||
extern const ServerSettingsUInt64 background_buffer_flush_schedule_pool_size;
|
||||
extern const ServerSettingsUInt64 background_common_pool_size;
|
||||
extern const ServerSettingsUInt64 background_distributed_schedule_pool_size;
|
||||
extern const ServerSettingsUInt64 background_fetches_pool_size;
|
||||
extern const ServerSettingsFloat background_merges_mutations_concurrency_ratio;
|
||||
extern const ServerSettingsString background_merges_mutations_scheduling_policy;
|
||||
extern const ServerSettingsUInt64 background_message_broker_schedule_pool_size;
|
||||
extern const ServerSettingsUInt64 background_move_pool_size;
|
||||
extern const ServerSettingsUInt64 background_pool_size;
|
||||
extern const ServerSettingsUInt64 background_schedule_pool_size;
|
||||
extern const ServerSettingsUInt64 backups_io_thread_pool_queue_size;
|
||||
extern const ServerSettingsDouble cache_size_to_ram_max_ratio;
|
||||
extern const ServerSettingsDouble cannot_allocate_thread_fault_injection_probability;
|
||||
extern const ServerSettingsUInt64 cgroups_memory_usage_observer_wait_time;
|
||||
extern const ServerSettingsUInt64 compiled_expression_cache_elements_size;
|
||||
extern const ServerSettingsUInt64 compiled_expression_cache_size;
|
||||
extern const ServerSettingsUInt64 concurrent_threads_soft_limit_num;
|
||||
extern const ServerSettingsUInt64 concurrent_threads_soft_limit_ratio_to_cores;
|
||||
extern const ServerSettingsUInt64 config_reload_interval_ms;
|
||||
extern const ServerSettingsUInt64 database_catalog_drop_table_concurrency;
|
||||
extern const ServerSettingsString default_database;
|
||||
extern const ServerSettingsBool disable_internal_dns_cache;
|
||||
extern const ServerSettingsUInt64 disk_connections_soft_limit;
|
||||
extern const ServerSettingsUInt64 disk_connections_store_limit;
|
||||
extern const ServerSettingsUInt64 disk_connections_warn_limit;
|
||||
extern const ServerSettingsBool dns_allow_resolve_names_to_ipv4;
|
||||
extern const ServerSettingsBool dns_allow_resolve_names_to_ipv6;
|
||||
extern const ServerSettingsUInt64 dns_cache_max_entries;
|
||||
extern const ServerSettingsInt32 dns_cache_update_period;
|
||||
extern const ServerSettingsUInt32 dns_max_consecutive_failures;
|
||||
extern const ServerSettingsBool enable_azure_sdk_logging;
|
||||
extern const ServerSettingsBool format_alter_operations_with_parentheses;
|
||||
extern const ServerSettingsUInt64 global_profiler_cpu_time_period_ns;
|
||||
extern const ServerSettingsUInt64 global_profiler_real_time_period_ns;
|
||||
extern const ServerSettingsDouble gwp_asan_force_sample_probability;
|
||||
extern const ServerSettingsUInt64 http_connections_soft_limit;
|
||||
extern const ServerSettingsUInt64 http_connections_store_limit;
|
||||
extern const ServerSettingsUInt64 http_connections_warn_limit;
|
||||
extern const ServerSettingsString index_mark_cache_policy;
|
||||
extern const ServerSettingsUInt64 index_mark_cache_size;
|
||||
extern const ServerSettingsDouble index_mark_cache_size_ratio;
|
||||
extern const ServerSettingsString index_uncompressed_cache_policy;
|
||||
extern const ServerSettingsUInt64 index_uncompressed_cache_size;
|
||||
extern const ServerSettingsDouble index_uncompressed_cache_size_ratio;
|
||||
extern const ServerSettingsUInt64 io_thread_pool_queue_size;
|
||||
extern const ServerSettingsSeconds keep_alive_timeout;
|
||||
extern const ServerSettingsString mark_cache_policy;
|
||||
extern const ServerSettingsUInt64 mark_cache_size;
|
||||
extern const ServerSettingsDouble mark_cache_size_ratio;
|
||||
extern const ServerSettingsUInt64 max_active_parts_loading_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_backups_io_thread_pool_free_size;
|
||||
extern const ServerSettingsUInt64 max_backups_io_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_concurrent_insert_queries;
|
||||
extern const ServerSettingsUInt64 max_concurrent_queries;
|
||||
extern const ServerSettingsUInt64 max_concurrent_select_queries;
|
||||
extern const ServerSettingsInt32 max_connections;
|
||||
extern const ServerSettingsUInt64 max_database_num_to_warn;
|
||||
extern const ServerSettingsUInt32 max_database_replicated_create_table_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_dictionary_num_to_warn;
|
||||
extern const ServerSettingsUInt64 max_io_thread_pool_free_size;
|
||||
extern const ServerSettingsUInt64 max_io_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_keep_alive_requests;
|
||||
extern const ServerSettingsUInt64 max_outdated_parts_loading_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_partition_size_to_drop;
|
||||
extern const ServerSettingsUInt64 max_part_num_to_warn;
|
||||
extern const ServerSettingsUInt64 max_parts_cleaning_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_server_memory_usage;
|
||||
extern const ServerSettingsDouble max_server_memory_usage_to_ram_ratio;
|
||||
extern const ServerSettingsUInt64 max_table_num_to_warn;
|
||||
extern const ServerSettingsUInt64 max_table_size_to_drop;
|
||||
extern const ServerSettingsUInt64 max_temporary_data_on_disk_size;
|
||||
extern const ServerSettingsUInt64 max_thread_pool_free_size;
|
||||
extern const ServerSettingsUInt64 max_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_unexpected_parts_loading_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_view_num_to_warn;
|
||||
extern const ServerSettingsUInt64 max_waiting_queries;
|
||||
extern const ServerSettingsUInt64 memory_worker_period_ms;
|
||||
extern const ServerSettingsUInt64 merges_mutations_memory_usage_soft_limit;
|
||||
extern const ServerSettingsDouble merges_mutations_memory_usage_to_ram_ratio;
|
||||
extern const ServerSettingsString merge_workload;
|
||||
extern const ServerSettingsUInt64 mmap_cache_size;
|
||||
extern const ServerSettingsString mutation_workload;
|
||||
extern const ServerSettingsUInt64 page_cache_chunk_size;
|
||||
extern const ServerSettingsUInt64 page_cache_mmap_size;
|
||||
extern const ServerSettingsUInt64 page_cache_size;
|
||||
extern const ServerSettingsBool page_cache_use_madv_free;
|
||||
extern const ServerSettingsBool page_cache_use_transparent_huge_pages;
|
||||
extern const ServerSettingsBool prepare_system_log_tables_on_startup;
|
||||
extern const ServerSettingsBool show_addresses_in_stack_traces;
|
||||
extern const ServerSettingsBool shutdown_wait_backups_and_restores;
|
||||
extern const ServerSettingsUInt64 shutdown_wait_unfinished;
|
||||
extern const ServerSettingsBool shutdown_wait_unfinished_queries;
|
||||
extern const ServerSettingsUInt64 storage_connections_soft_limit;
|
||||
extern const ServerSettingsUInt64 storage_connections_store_limit;
|
||||
extern const ServerSettingsUInt64 storage_connections_warn_limit;
|
||||
extern const ServerSettingsUInt64 tables_loader_background_pool_size;
|
||||
extern const ServerSettingsUInt64 tables_loader_foreground_pool_size;
|
||||
extern const ServerSettingsString temporary_data_in_cache;
|
||||
extern const ServerSettingsUInt64 thread_pool_queue_size;
|
||||
extern const ServerSettingsString tmp_policy;
|
||||
extern const ServerSettingsUInt64 total_memory_profiler_sample_max_allocation_size;
|
||||
extern const ServerSettingsUInt64 total_memory_profiler_sample_min_allocation_size;
|
||||
extern const ServerSettingsUInt64 total_memory_profiler_step;
|
||||
extern const ServerSettingsDouble total_memory_tracker_sample_probability;
|
||||
extern const ServerSettingsString uncompressed_cache_policy;
|
||||
extern const ServerSettingsUInt64 uncompressed_cache_size;
|
||||
extern const ServerSettingsDouble uncompressed_cache_size_ratio;
|
||||
extern const ServerSettingsBool use_legacy_mongodb_integration;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -672,7 +789,7 @@ static void initializeAzureSDKLogger(
|
||||
[[ maybe_unused ]] int server_logs_level)
|
||||
{
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
if (!server_settings.enable_azure_sdk_logging)
|
||||
if (!server_settings[ServerSetting::enable_azure_sdk_logging])
|
||||
return;
|
||||
|
||||
using AzureLogsLevel = Azure::Core::Diagnostics::Logger::Level;
|
||||
@ -746,9 +863,9 @@ try
|
||||
ServerSettings server_settings;
|
||||
server_settings.loadSettingsFromConfig(config());
|
||||
|
||||
ASTAlterCommand::setFormatAlterCommandsWithParentheses(server_settings.format_alter_operations_with_parentheses);
|
||||
ASTAlterCommand::setFormatAlterCommandsWithParentheses(server_settings[ServerSetting::format_alter_operations_with_parentheses]);
|
||||
|
||||
StackTrace::setShowAddresses(server_settings.show_addresses_in_stack_traces);
|
||||
StackTrace::setShowAddresses(server_settings[ServerSetting::show_addresses_in_stack_traces]);
|
||||
|
||||
#if USE_HDFS
|
||||
/// This will point libhdfs3 to the right location for its config.
|
||||
@ -794,10 +911,10 @@ try
|
||||
registerInterpreters();
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
registerTableFunctions(server_settings.use_legacy_mongodb_integration);
|
||||
registerTableFunctions(server_settings[ServerSetting::use_legacy_mongodb_integration]);
|
||||
registerDatabases();
|
||||
registerStorages(server_settings.use_legacy_mongodb_integration);
|
||||
registerDictionaries(server_settings.use_legacy_mongodb_integration);
|
||||
registerStorages(server_settings[ServerSetting::use_legacy_mongodb_integration]);
|
||||
registerDictionaries(server_settings[ServerSetting::use_legacy_mongodb_integration]);
|
||||
registerDisks(/* global_skip_access_check= */ false);
|
||||
registerFormats();
|
||||
registerRemoteFileMetadatas();
|
||||
@ -893,37 +1010,37 @@ 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(
|
||||
server_settings.max_thread_pool_size,
|
||||
server_settings.max_thread_pool_free_size,
|
||||
server_settings.thread_pool_queue_size,
|
||||
has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0,
|
||||
has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0);
|
||||
server_settings[ServerSetting::max_thread_pool_size],
|
||||
server_settings[ServerSetting::max_thread_pool_free_size],
|
||||
server_settings[ServerSetting::thread_pool_queue_size],
|
||||
has_trace_collector ? server_settings[ServerSetting::global_profiler_real_time_period_ns] : 0,
|
||||
has_trace_collector ? server_settings[ServerSetting::global_profiler_cpu_time_period_ns] : 0);
|
||||
|
||||
if (has_trace_collector)
|
||||
{
|
||||
global_context->createTraceCollector();
|
||||
|
||||
/// Set up server-wide memory profiler (for total memory tracker).
|
||||
if (server_settings.total_memory_profiler_step)
|
||||
total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step);
|
||||
if (server_settings[ServerSetting::total_memory_profiler_step])
|
||||
total_memory_tracker.setProfilerStep(server_settings[ServerSetting::total_memory_profiler_step]);
|
||||
|
||||
if (server_settings.total_memory_tracker_sample_probability > 0.0)
|
||||
total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability);
|
||||
if (server_settings[ServerSetting::total_memory_tracker_sample_probability] > 0.0)
|
||||
total_memory_tracker.setSampleProbability(server_settings[ServerSetting::total_memory_tracker_sample_probability]);
|
||||
|
||||
if (server_settings.total_memory_profiler_sample_min_allocation_size)
|
||||
total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size);
|
||||
if (server_settings[ServerSetting::total_memory_profiler_sample_min_allocation_size])
|
||||
total_memory_tracker.setSampleMinAllocationSize(server_settings[ServerSetting::total_memory_profiler_sample_min_allocation_size]);
|
||||
|
||||
if (server_settings.total_memory_profiler_sample_max_allocation_size)
|
||||
total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size);
|
||||
if (server_settings[ServerSetting::total_memory_profiler_sample_max_allocation_size])
|
||||
total_memory_tracker.setSampleMaxAllocationSize(server_settings[ServerSetting::total_memory_profiler_sample_max_allocation_size]);
|
||||
}
|
||||
|
||||
Poco::ThreadPool server_pool(
|
||||
/* minCapacity */3,
|
||||
/* maxCapacity */server_settings.max_connections,
|
||||
/* maxCapacity */server_settings[ServerSetting::max_connections],
|
||||
/* idleTime */60,
|
||||
/* stackSize */POCO_THREAD_STACK_SIZE,
|
||||
server_settings.global_profiler_real_time_period_ns,
|
||||
server_settings.global_profiler_cpu_time_period_ns);
|
||||
server_settings[ServerSetting::global_profiler_real_time_period_ns],
|
||||
server_settings[ServerSetting::global_profiler_cpu_time_period_ns]);
|
||||
|
||||
std::mutex servers_lock;
|
||||
std::vector<ProtocolServerAdapter> servers;
|
||||
@ -937,13 +1054,13 @@ try
|
||||
LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds());
|
||||
});
|
||||
|
||||
MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms);
|
||||
MemoryWorker memory_worker(global_context->getServerSettings()[ServerSetting::memory_worker_period_ms]);
|
||||
|
||||
/// This object will periodically calculate some metrics.
|
||||
ServerAsynchronousMetrics async_metrics(
|
||||
global_context,
|
||||
server_settings.asynchronous_metrics_update_period_s,
|
||||
server_settings.asynchronous_heavy_metrics_update_period_s,
|
||||
server_settings[ServerSetting::asynchronous_metrics_update_period_s],
|
||||
server_settings[ServerSetting::asynchronous_heavy_metrics_update_period_s],
|
||||
[&]() -> std::vector<ProtocolServerMetrics>
|
||||
{
|
||||
std::vector<ProtocolServerMetrics> metrics;
|
||||
@ -996,7 +1113,7 @@ try
|
||||
LOG_INFO(log, "Closed all listening sockets.");
|
||||
|
||||
if (current_connections > 0)
|
||||
current_connections = waitServersToFinish(servers_to_start_before_tables, servers_lock, server_settings.shutdown_wait_unfinished);
|
||||
current_connections = waitServersToFinish(servers_to_start_before_tables, servers_lock, server_settings[ServerSetting::shutdown_wait_unfinished]);
|
||||
|
||||
if (current_connections)
|
||||
LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections);
|
||||
@ -1033,47 +1150,47 @@ try
|
||||
#endif
|
||||
|
||||
getIOThreadPool().initialize(
|
||||
server_settings.max_io_thread_pool_size,
|
||||
server_settings.max_io_thread_pool_free_size,
|
||||
server_settings.io_thread_pool_queue_size);
|
||||
server_settings[ServerSetting::max_io_thread_pool_size],
|
||||
server_settings[ServerSetting::max_io_thread_pool_free_size],
|
||||
server_settings[ServerSetting::io_thread_pool_queue_size]);
|
||||
|
||||
getBackupsIOThreadPool().initialize(
|
||||
server_settings.max_backups_io_thread_pool_size,
|
||||
server_settings.max_backups_io_thread_pool_free_size,
|
||||
server_settings.backups_io_thread_pool_queue_size);
|
||||
server_settings[ServerSetting::max_backups_io_thread_pool_size],
|
||||
server_settings[ServerSetting::max_backups_io_thread_pool_free_size],
|
||||
server_settings[ServerSetting::backups_io_thread_pool_queue_size]);
|
||||
|
||||
getActivePartsLoadingThreadPool().initialize(
|
||||
server_settings.max_active_parts_loading_thread_pool_size,
|
||||
server_settings[ServerSetting::max_active_parts_loading_thread_pool_size],
|
||||
0, // We don't need any threads once all the parts will be loaded
|
||||
server_settings.max_active_parts_loading_thread_pool_size);
|
||||
server_settings[ServerSetting::max_active_parts_loading_thread_pool_size]);
|
||||
|
||||
getOutdatedPartsLoadingThreadPool().initialize(
|
||||
server_settings.max_outdated_parts_loading_thread_pool_size,
|
||||
server_settings[ServerSetting::max_outdated_parts_loading_thread_pool_size],
|
||||
0, // We don't need any threads once all the parts will be loaded
|
||||
server_settings.max_outdated_parts_loading_thread_pool_size);
|
||||
server_settings[ServerSetting::max_outdated_parts_loading_thread_pool_size]);
|
||||
|
||||
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
|
||||
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
|
||||
server_settings.max_active_parts_loading_thread_pool_size
|
||||
server_settings[ServerSetting::max_active_parts_loading_thread_pool_size]
|
||||
);
|
||||
|
||||
getUnexpectedPartsLoadingThreadPool().initialize(
|
||||
server_settings.max_unexpected_parts_loading_thread_pool_size,
|
||||
server_settings[ServerSetting::max_unexpected_parts_loading_thread_pool_size],
|
||||
0, // We don't need any threads once all the parts will be loaded
|
||||
server_settings.max_unexpected_parts_loading_thread_pool_size);
|
||||
server_settings[ServerSetting::max_unexpected_parts_loading_thread_pool_size]);
|
||||
|
||||
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
|
||||
getUnexpectedPartsLoadingThreadPool().setMaxTurboThreads(
|
||||
server_settings.max_active_parts_loading_thread_pool_size
|
||||
server_settings[ServerSetting::max_active_parts_loading_thread_pool_size]
|
||||
);
|
||||
|
||||
getPartsCleaningThreadPool().initialize(
|
||||
server_settings.max_parts_cleaning_thread_pool_size,
|
||||
server_settings[ServerSetting::max_parts_cleaning_thread_pool_size],
|
||||
0, // We don't need any threads one all the parts will be deleted
|
||||
server_settings.max_parts_cleaning_thread_pool_size);
|
||||
server_settings[ServerSetting::max_parts_cleaning_thread_pool_size]);
|
||||
|
||||
auto max_database_replicated_create_table_thread_pool_size = server_settings.max_database_replicated_create_table_thread_pool_size
|
||||
? server_settings.max_database_replicated_create_table_thread_pool_size
|
||||
auto max_database_replicated_create_table_thread_pool_size = server_settings[ServerSetting::max_database_replicated_create_table_thread_pool_size]
|
||||
? server_settings[ServerSetting::max_database_replicated_create_table_thread_pool_size]
|
||||
: getNumberOfCPUCoresToUse();
|
||||
getDatabaseReplicatedCreateTablesThreadPool().initialize(
|
||||
max_database_replicated_create_table_thread_pool_size,
|
||||
@ -1081,9 +1198,9 @@ try
|
||||
max_database_replicated_create_table_thread_pool_size);
|
||||
|
||||
getDatabaseCatalogDropTablesThreadPool().initialize(
|
||||
server_settings.database_catalog_drop_table_concurrency,
|
||||
server_settings[ServerSetting::database_catalog_drop_table_concurrency],
|
||||
0, // We don't need any threads if there are no DROP queries.
|
||||
server_settings.database_catalog_drop_table_concurrency);
|
||||
server_settings[ServerSetting::database_catalog_drop_table_concurrency]);
|
||||
|
||||
/// Initialize global local cache for remote filesystem.
|
||||
if (config().has("local_cache_for_remote_fs"))
|
||||
@ -1320,18 +1437,18 @@ try
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone());
|
||||
|
||||
/// Storage with temporary data for processing of heavy queries.
|
||||
if (!server_settings.tmp_policy.value.empty())
|
||||
if (!server_settings[ServerSetting::tmp_policy].value.empty())
|
||||
{
|
||||
global_context->setTemporaryStoragePolicy(server_settings.tmp_policy, server_settings.max_temporary_data_on_disk_size);
|
||||
global_context->setTemporaryStoragePolicy(server_settings[ServerSetting::tmp_policy], server_settings[ServerSetting::max_temporary_data_on_disk_size]);
|
||||
}
|
||||
else if (!server_settings.temporary_data_in_cache.value.empty())
|
||||
else if (!server_settings[ServerSetting::temporary_data_in_cache].value.empty())
|
||||
{
|
||||
global_context->setTemporaryStorageInCache(server_settings.temporary_data_in_cache, server_settings.max_temporary_data_on_disk_size);
|
||||
global_context->setTemporaryStorageInCache(server_settings[ServerSetting::temporary_data_in_cache], server_settings[ServerSetting::max_temporary_data_on_disk_size]);
|
||||
}
|
||||
else
|
||||
{
|
||||
std::string temporary_path = config().getString("tmp_path", path / "tmp/");
|
||||
global_context->setTemporaryStoragePath(temporary_path, server_settings.max_temporary_data_on_disk_size);
|
||||
global_context->setTemporaryStoragePath(temporary_path, server_settings[ServerSetting::max_temporary_data_on_disk_size]);
|
||||
}
|
||||
|
||||
/** Directory with 'flags': files indicating temporary settings for the server set by system administrator.
|
||||
@ -1420,11 +1537,11 @@ try
|
||||
|
||||
/// Set up caches.
|
||||
|
||||
const size_t max_cache_size = static_cast<size_t>(physical_server_memory * server_settings.cache_size_to_ram_max_ratio);
|
||||
const size_t max_cache_size = static_cast<size_t>(physical_server_memory * server_settings[ServerSetting::cache_size_to_ram_max_ratio]);
|
||||
|
||||
String uncompressed_cache_policy = server_settings.uncompressed_cache_policy;
|
||||
size_t uncompressed_cache_size = server_settings.uncompressed_cache_size;
|
||||
double uncompressed_cache_size_ratio = server_settings.uncompressed_cache_size_ratio;
|
||||
String uncompressed_cache_policy = server_settings[ServerSetting::uncompressed_cache_policy];
|
||||
size_t uncompressed_cache_size = server_settings[ServerSetting::uncompressed_cache_size];
|
||||
double uncompressed_cache_size_ratio = server_settings[ServerSetting::uncompressed_cache_size_ratio];
|
||||
if (uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
uncompressed_cache_size = max_cache_size;
|
||||
@ -1432,9 +1549,9 @@ try
|
||||
}
|
||||
global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size, uncompressed_cache_size_ratio);
|
||||
|
||||
String mark_cache_policy = server_settings.mark_cache_policy;
|
||||
size_t mark_cache_size = server_settings.mark_cache_size;
|
||||
double mark_cache_size_ratio = server_settings.mark_cache_size_ratio;
|
||||
String mark_cache_policy = server_settings[ServerSetting::mark_cache_policy];
|
||||
size_t mark_cache_size = server_settings[ServerSetting::mark_cache_size];
|
||||
double mark_cache_size_ratio = server_settings[ServerSetting::mark_cache_size_ratio];
|
||||
if (mark_cache_size > max_cache_size)
|
||||
{
|
||||
mark_cache_size = max_cache_size;
|
||||
@ -1442,16 +1559,16 @@ try
|
||||
}
|
||||
global_context->setMarkCache(mark_cache_policy, mark_cache_size, mark_cache_size_ratio);
|
||||
|
||||
size_t page_cache_size = server_settings.page_cache_size;
|
||||
size_t page_cache_size = server_settings[ServerSetting::page_cache_size];
|
||||
if (page_cache_size != 0)
|
||||
global_context->setPageCache(
|
||||
server_settings.page_cache_chunk_size, server_settings.page_cache_mmap_size,
|
||||
page_cache_size, server_settings.page_cache_use_madv_free,
|
||||
server_settings.page_cache_use_transparent_huge_pages);
|
||||
server_settings[ServerSetting::page_cache_chunk_size], server_settings[ServerSetting::page_cache_mmap_size],
|
||||
page_cache_size, server_settings[ServerSetting::page_cache_use_madv_free],
|
||||
server_settings[ServerSetting::page_cache_use_transparent_huge_pages]);
|
||||
|
||||
String index_uncompressed_cache_policy = server_settings.index_uncompressed_cache_policy;
|
||||
size_t index_uncompressed_cache_size = server_settings.index_uncompressed_cache_size;
|
||||
double index_uncompressed_cache_size_ratio = server_settings.index_uncompressed_cache_size_ratio;
|
||||
String index_uncompressed_cache_policy = server_settings[ServerSetting::index_uncompressed_cache_policy];
|
||||
size_t index_uncompressed_cache_size = server_settings[ServerSetting::index_uncompressed_cache_size];
|
||||
double index_uncompressed_cache_size_ratio = server_settings[ServerSetting::index_uncompressed_cache_size_ratio];
|
||||
if (index_uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
index_uncompressed_cache_size = max_cache_size;
|
||||
@ -1459,9 +1576,9 @@ try
|
||||
}
|
||||
global_context->setIndexUncompressedCache(index_uncompressed_cache_policy, index_uncompressed_cache_size, index_uncompressed_cache_size_ratio);
|
||||
|
||||
String index_mark_cache_policy = server_settings.index_mark_cache_policy;
|
||||
size_t index_mark_cache_size = server_settings.index_mark_cache_size;
|
||||
double index_mark_cache_size_ratio = server_settings.index_mark_cache_size_ratio;
|
||||
String index_mark_cache_policy = server_settings[ServerSetting::index_mark_cache_policy];
|
||||
size_t index_mark_cache_size = server_settings[ServerSetting::index_mark_cache_size];
|
||||
double index_mark_cache_size_ratio = server_settings[ServerSetting::index_mark_cache_size_ratio];
|
||||
if (index_mark_cache_size > max_cache_size)
|
||||
{
|
||||
index_mark_cache_size = max_cache_size;
|
||||
@ -1469,7 +1586,7 @@ try
|
||||
}
|
||||
global_context->setIndexMarkCache(index_mark_cache_policy, index_mark_cache_size, index_mark_cache_size_ratio);
|
||||
|
||||
size_t mmap_cache_size = server_settings.mmap_cache_size;
|
||||
size_t mmap_cache_size = server_settings[ServerSetting::mmap_cache_size];
|
||||
if (mmap_cache_size > max_cache_size)
|
||||
{
|
||||
mmap_cache_size = max_cache_size;
|
||||
@ -1489,8 +1606,8 @@ try
|
||||
global_context->setQueryCache(query_cache_max_size_in_bytes, query_cache_max_entries, query_cache_query_cache_max_entry_size_in_bytes, query_cache_max_entry_size_in_rows);
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
size_t compiled_expression_cache_max_size_in_bytes = server_settings.compiled_expression_cache_size;
|
||||
size_t compiled_expression_cache_max_elements = server_settings.compiled_expression_cache_elements_size;
|
||||
size_t compiled_expression_cache_max_size_in_bytes = server_settings[ServerSetting::compiled_expression_cache_size];
|
||||
size_t compiled_expression_cache_max_elements = server_settings[ServerSetting::compiled_expression_cache_elements_size];
|
||||
CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements);
|
||||
#endif
|
||||
|
||||
@ -1509,7 +1626,7 @@ try
|
||||
std::optional<CgroupsMemoryUsageObserver> cgroups_memory_usage_observer;
|
||||
try
|
||||
{
|
||||
auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time;
|
||||
auto wait_time = server_settings[ServerSetting::cgroups_memory_usage_observer_wait_time];
|
||||
if (wait_time != 0)
|
||||
cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time));
|
||||
}
|
||||
@ -1552,8 +1669,8 @@ try
|
||||
ServerSettings new_server_settings;
|
||||
new_server_settings.loadSettingsFromConfig(*config);
|
||||
|
||||
size_t max_server_memory_usage = new_server_settings.max_server_memory_usage;
|
||||
double max_server_memory_usage_to_ram_ratio = new_server_settings.max_server_memory_usage_to_ram_ratio;
|
||||
size_t max_server_memory_usage = new_server_settings[ServerSetting::max_server_memory_usage];
|
||||
double max_server_memory_usage_to_ram_ratio = new_server_settings[ServerSetting::max_server_memory_usage_to_ram_ratio];
|
||||
|
||||
size_t current_physical_server_memory = getMemoryAmount(); /// With cgroups, the amount of memory available to the server can be changed dynamically.
|
||||
size_t default_max_server_memory_usage = static_cast<size_t>(current_physical_server_memory * max_server_memory_usage_to_ram_ratio);
|
||||
@ -1583,9 +1700,9 @@ try
|
||||
total_memory_tracker.setDescription("(total)");
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
|
||||
size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit;
|
||||
size_t merges_mutations_memory_usage_soft_limit = new_server_settings[ServerSetting::merges_mutations_memory_usage_soft_limit];
|
||||
|
||||
size_t default_merges_mutations_server_memory_usage = static_cast<size_t>(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
||||
size_t default_merges_mutations_server_memory_usage = static_cast<size_t>(current_physical_server_memory * new_server_settings[ServerSetting::merges_mutations_memory_usage_to_ram_ratio]);
|
||||
if (merges_mutations_memory_usage_soft_limit == 0)
|
||||
{
|
||||
merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage;
|
||||
@ -1593,7 +1710,7 @@ try
|
||||
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)",
|
||||
formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit),
|
||||
formatReadableSizeWithBinarySuffix(current_physical_server_memory),
|
||||
new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
||||
new_server_settings[ServerSetting::merges_mutations_memory_usage_to_ram_ratio]);
|
||||
}
|
||||
else if (merges_mutations_memory_usage_soft_limit > default_merges_mutations_server_memory_usage)
|
||||
{
|
||||
@ -1602,7 +1719,7 @@ try
|
||||
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)",
|
||||
formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit),
|
||||
formatReadableSizeWithBinarySuffix(current_physical_server_memory),
|
||||
new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
||||
new_server_settings[ServerSetting::merges_mutations_memory_usage_to_ram_ratio]);
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Merges and mutations memory limit is set to {}",
|
||||
@ -1635,31 +1752,31 @@ try
|
||||
global_context->setRemoteHostFilter(*config);
|
||||
global_context->setHTTPHeaderFilter(*config);
|
||||
|
||||
global_context->setMaxTableSizeToDrop(new_server_settings.max_table_size_to_drop);
|
||||
global_context->setMaxPartitionSizeToDrop(new_server_settings.max_partition_size_to_drop);
|
||||
global_context->setMaxTableNumToWarn(new_server_settings.max_table_num_to_warn);
|
||||
global_context->setMaxViewNumToWarn(new_server_settings.max_view_num_to_warn);
|
||||
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);
|
||||
global_context->setMaxTableSizeToDrop(new_server_settings[ServerSetting::max_table_size_to_drop]);
|
||||
global_context->setMaxPartitionSizeToDrop(new_server_settings[ServerSetting::max_partition_size_to_drop]);
|
||||
global_context->setMaxTableNumToWarn(new_server_settings[ServerSetting::max_table_num_to_warn]);
|
||||
global_context->setMaxViewNumToWarn(new_server_settings[ServerSetting::max_view_num_to_warn]);
|
||||
global_context->setMaxDictionaryNumToWarn(new_server_settings[ServerSetting::max_dictionary_num_to_warn]);
|
||||
global_context->setMaxDatabaseNumToWarn(new_server_settings[ServerSetting::max_database_num_to_warn]);
|
||||
global_context->setMaxPartNumToWarn(new_server_settings[ServerSetting::max_part_num_to_warn]);
|
||||
/// Only for system.server_settings
|
||||
global_context->setConfigReloaderInterval(new_server_settings.config_reload_interval_ms);
|
||||
global_context->setConfigReloaderInterval(new_server_settings[ServerSetting::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)
|
||||
concurrent_threads_soft_limit = new_server_settings.concurrent_threads_soft_limit_num;
|
||||
if (new_server_settings.concurrent_threads_soft_limit_ratio_to_cores > 0)
|
||||
if (new_server_settings[ServerSetting::concurrent_threads_soft_limit_num] > 0 && new_server_settings[ServerSetting::concurrent_threads_soft_limit_num] < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = new_server_settings[ServerSetting::concurrent_threads_soft_limit_num];
|
||||
if (new_server_settings[ServerSetting::concurrent_threads_soft_limit_ratio_to_cores] > 0)
|
||||
{
|
||||
auto value = new_server_settings.concurrent_threads_soft_limit_ratio_to_cores * getNumberOfCPUCoresToUse();
|
||||
auto value = new_server_settings[ServerSetting::concurrent_threads_soft_limit_ratio_to_cores] * getNumberOfCPUCoresToUse();
|
||||
if (value > 0 && value < concurrent_threads_soft_limit)
|
||||
concurrent_threads_soft_limit = value;
|
||||
}
|
||||
ConcurrencyControl::instance().setMaxConcurrency(concurrent_threads_soft_limit);
|
||||
|
||||
global_context->getProcessList().setMaxSize(new_server_settings.max_concurrent_queries);
|
||||
global_context->getProcessList().setMaxInsertQueriesAmount(new_server_settings.max_concurrent_insert_queries);
|
||||
global_context->getProcessList().setMaxSelectQueriesAmount(new_server_settings.max_concurrent_select_queries);
|
||||
global_context->getProcessList().setMaxWaitingQueriesAmount(new_server_settings.max_waiting_queries);
|
||||
global_context->getProcessList().setMaxSize(new_server_settings[ServerSetting::max_concurrent_queries]);
|
||||
global_context->getProcessList().setMaxInsertQueriesAmount(new_server_settings[ServerSetting::max_concurrent_insert_queries]);
|
||||
global_context->getProcessList().setMaxSelectQueriesAmount(new_server_settings[ServerSetting::max_concurrent_select_queries]);
|
||||
global_context->getProcessList().setMaxWaitingQueriesAmount(new_server_settings[ServerSetting::max_waiting_queries]);
|
||||
|
||||
if (config->has("keeper_server"))
|
||||
global_context->updateKeeperConfiguration(*config);
|
||||
@ -1670,72 +1787,72 @@ try
|
||||
/// This is done for backward compatibility.
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = new_server_settings.background_pool_size;
|
||||
auto new_ratio = new_server_settings.background_merges_mutations_concurrency_ratio;
|
||||
auto new_pool_size = new_server_settings[ServerSetting::background_pool_size];
|
||||
auto new_ratio = new_server_settings[ServerSetting::background_merges_mutations_concurrency_ratio];
|
||||
global_context->getMergeMutateExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, static_cast<size_t>(new_pool_size * new_ratio));
|
||||
global_context->getMergeMutateExecutor()->updateSchedulingPolicy(new_server_settings.background_merges_mutations_scheduling_policy.toString());
|
||||
global_context->getMergeMutateExecutor()->updateSchedulingPolicy(new_server_settings[ServerSetting::background_merges_mutations_scheduling_policy].toString());
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = new_server_settings.background_move_pool_size;
|
||||
auto new_pool_size = new_server_settings[ServerSetting::background_move_pool_size];
|
||||
global_context->getMovesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = new_server_settings.background_fetches_pool_size;
|
||||
auto new_pool_size = new_server_settings[ServerSetting::background_fetches_pool_size];
|
||||
global_context->getFetchesExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
if (global_context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
auto new_pool_size = new_server_settings.background_common_pool_size;
|
||||
auto new_pool_size = new_server_settings[ServerSetting::background_common_pool_size];
|
||||
global_context->getCommonExecutor()->increaseThreadsAndMaxTasksCount(new_pool_size, new_pool_size);
|
||||
}
|
||||
|
||||
global_context->getBufferFlushSchedulePool().increaseThreadsCount(new_server_settings.background_buffer_flush_schedule_pool_size);
|
||||
global_context->getSchedulePool().increaseThreadsCount(new_server_settings.background_schedule_pool_size);
|
||||
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(new_server_settings.background_message_broker_schedule_pool_size);
|
||||
global_context->getDistributedSchedulePool().increaseThreadsCount(new_server_settings.background_distributed_schedule_pool_size);
|
||||
global_context->getBufferFlushSchedulePool().increaseThreadsCount(new_server_settings[ServerSetting::background_buffer_flush_schedule_pool_size]);
|
||||
global_context->getSchedulePool().increaseThreadsCount(new_server_settings[ServerSetting::background_schedule_pool_size]);
|
||||
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(new_server_settings[ServerSetting::background_message_broker_schedule_pool_size]);
|
||||
global_context->getDistributedSchedulePool().increaseThreadsCount(new_server_settings[ServerSetting::background_distributed_schedule_pool_size]);
|
||||
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, new_server_settings.tables_loader_foreground_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, new_server_settings.tables_loader_background_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, new_server_settings.tables_loader_background_pool_size);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderForegroundPoolId, new_server_settings[ServerSetting::tables_loader_foreground_pool_size]);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundLoadPoolId, new_server_settings[ServerSetting::tables_loader_background_pool_size]);
|
||||
global_context->getAsyncLoader().setMaxThreads(TablesLoaderBackgroundStartupPoolId, new_server_settings[ServerSetting::tables_loader_background_pool_size]);
|
||||
|
||||
getIOThreadPool().reloadConfiguration(
|
||||
new_server_settings.max_io_thread_pool_size,
|
||||
new_server_settings.max_io_thread_pool_free_size,
|
||||
new_server_settings.io_thread_pool_queue_size);
|
||||
new_server_settings[ServerSetting::max_io_thread_pool_size],
|
||||
new_server_settings[ServerSetting::max_io_thread_pool_free_size],
|
||||
new_server_settings[ServerSetting::io_thread_pool_queue_size]);
|
||||
|
||||
getBackupsIOThreadPool().reloadConfiguration(
|
||||
new_server_settings.max_backups_io_thread_pool_size,
|
||||
new_server_settings.max_backups_io_thread_pool_free_size,
|
||||
new_server_settings.backups_io_thread_pool_queue_size);
|
||||
new_server_settings[ServerSetting::max_backups_io_thread_pool_size],
|
||||
new_server_settings[ServerSetting::max_backups_io_thread_pool_free_size],
|
||||
new_server_settings[ServerSetting::backups_io_thread_pool_queue_size]);
|
||||
|
||||
getActivePartsLoadingThreadPool().reloadConfiguration(
|
||||
new_server_settings.max_active_parts_loading_thread_pool_size,
|
||||
new_server_settings[ServerSetting::max_active_parts_loading_thread_pool_size],
|
||||
0, // We don't need any threads once all the parts will be loaded
|
||||
new_server_settings.max_active_parts_loading_thread_pool_size);
|
||||
new_server_settings[ServerSetting::max_active_parts_loading_thread_pool_size]);
|
||||
|
||||
getOutdatedPartsLoadingThreadPool().reloadConfiguration(
|
||||
new_server_settings.max_outdated_parts_loading_thread_pool_size,
|
||||
new_server_settings[ServerSetting::max_outdated_parts_loading_thread_pool_size],
|
||||
0, // We don't need any threads once all the parts will be loaded
|
||||
new_server_settings.max_outdated_parts_loading_thread_pool_size);
|
||||
new_server_settings[ServerSetting::max_outdated_parts_loading_thread_pool_size]);
|
||||
|
||||
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
|
||||
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
|
||||
new_server_settings.max_active_parts_loading_thread_pool_size
|
||||
new_server_settings[ServerSetting::max_active_parts_loading_thread_pool_size]
|
||||
);
|
||||
|
||||
getPartsCleaningThreadPool().reloadConfiguration(
|
||||
new_server_settings.max_parts_cleaning_thread_pool_size,
|
||||
new_server_settings[ServerSetting::max_parts_cleaning_thread_pool_size],
|
||||
0, // We don't need any threads one all the parts will be deleted
|
||||
new_server_settings.max_parts_cleaning_thread_pool_size);
|
||||
new_server_settings[ServerSetting::max_parts_cleaning_thread_pool_size]);
|
||||
|
||||
|
||||
global_context->setMergeWorkload(new_server_settings.merge_workload);
|
||||
global_context->setMutationWorkload(new_server_settings.mutation_workload);
|
||||
global_context->setMergeWorkload(new_server_settings[ServerSetting::merge_workload]);
|
||||
global_context->setMutationWorkload(new_server_settings[ServerSetting::mutation_workload]);
|
||||
|
||||
if (config->has("resources"))
|
||||
{
|
||||
@ -1780,28 +1897,28 @@ try
|
||||
|
||||
HTTPConnectionPools::instance().setLimits(
|
||||
HTTPConnectionPools::Limits{
|
||||
new_server_settings.disk_connections_soft_limit,
|
||||
new_server_settings.disk_connections_warn_limit,
|
||||
new_server_settings.disk_connections_store_limit,
|
||||
new_server_settings[ServerSetting::disk_connections_soft_limit],
|
||||
new_server_settings[ServerSetting::disk_connections_warn_limit],
|
||||
new_server_settings[ServerSetting::disk_connections_store_limit],
|
||||
},
|
||||
HTTPConnectionPools::Limits{
|
||||
new_server_settings.storage_connections_soft_limit,
|
||||
new_server_settings.storage_connections_warn_limit,
|
||||
new_server_settings.storage_connections_store_limit,
|
||||
new_server_settings[ServerSetting::storage_connections_soft_limit],
|
||||
new_server_settings[ServerSetting::storage_connections_warn_limit],
|
||||
new_server_settings[ServerSetting::storage_connections_store_limit],
|
||||
},
|
||||
HTTPConnectionPools::Limits{
|
||||
new_server_settings.http_connections_soft_limit,
|
||||
new_server_settings.http_connections_warn_limit,
|
||||
new_server_settings.http_connections_store_limit,
|
||||
new_server_settings[ServerSetting::http_connections_soft_limit],
|
||||
new_server_settings[ServerSetting::http_connections_warn_limit],
|
||||
new_server_settings[ServerSetting::http_connections_store_limit],
|
||||
});
|
||||
|
||||
DNSResolver::instance().setFilterSettings(new_server_settings.dns_allow_resolve_names_to_ipv4, new_server_settings.dns_allow_resolve_names_to_ipv6);
|
||||
DNSResolver::instance().setFilterSettings(new_server_settings[ServerSetting::dns_allow_resolve_names_to_ipv4], new_server_settings[ServerSetting::dns_allow_resolve_names_to_ipv6]);
|
||||
|
||||
if (global_context->isServerCompletelyStarted())
|
||||
CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings.cannot_allocate_thread_fault_injection_probability);
|
||||
CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings[ServerSetting::cannot_allocate_thread_fault_injection_probability]);
|
||||
|
||||
#if USE_GWP_ASAN
|
||||
GWPAsan::setForceSampleProbability(new_server_settings.gwp_asan_force_sample_probability);
|
||||
GWPAsan::setForceSampleProbability(new_server_settings[ServerSetting::gwp_asan_force_sample_probability]);
|
||||
#endif
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::MainConfigLoads);
|
||||
@ -1997,7 +2114,7 @@ try
|
||||
});
|
||||
|
||||
/// Limit on total number of concurrently executed queries.
|
||||
global_context->getProcessList().setMaxSize(server_settings.max_concurrent_queries);
|
||||
global_context->getProcessList().setMaxSize(server_settings[ServerSetting::max_concurrent_queries]);
|
||||
|
||||
/// Load global settings from default_profile and system_profile.
|
||||
global_context->setDefaultProfiles(config());
|
||||
@ -2006,12 +2123,12 @@ try
|
||||
/// This is needed to load proper values of background_pool_size etc.
|
||||
global_context->initializeBackgroundExecutorsIfNeeded();
|
||||
|
||||
if (server_settings.async_insert_threads)
|
||||
if (server_settings[ServerSetting::async_insert_threads])
|
||||
{
|
||||
global_context->setAsynchronousInsertQueue(std::make_shared<AsynchronousInsertQueue>(
|
||||
global_context,
|
||||
server_settings.async_insert_threads,
|
||||
server_settings.async_insert_queue_flush_on_shutdown));
|
||||
server_settings[ServerSetting::async_insert_threads],
|
||||
server_settings[ServerSetting::async_insert_queue_flush_on_shutdown]));
|
||||
}
|
||||
|
||||
/// Set path for format schema files
|
||||
@ -2047,7 +2164,7 @@ try
|
||||
/// context is destroyed.
|
||||
/// In addition this object has to be created before the loading of the tables.
|
||||
std::unique_ptr<DNSCacheUpdater> dns_cache_updater;
|
||||
if (server_settings.disable_internal_dns_cache)
|
||||
if (server_settings[ServerSetting::disable_internal_dns_cache])
|
||||
{
|
||||
/// Disable DNS caching at all
|
||||
DNSResolver::instance().setDisableCacheFlag();
|
||||
@ -2055,11 +2172,11 @@ try
|
||||
}
|
||||
else
|
||||
{
|
||||
DNSResolver::instance().setCacheMaxEntries(server_settings.dns_cache_max_entries);
|
||||
DNSResolver::instance().setCacheMaxEntries(server_settings[ServerSetting::dns_cache_max_entries]);
|
||||
|
||||
/// Initialize a watcher periodically updating DNS cache
|
||||
dns_cache_updater = std::make_unique<DNSCacheUpdater>(
|
||||
global_context, server_settings.dns_cache_update_period, server_settings.dns_max_consecutive_failures);
|
||||
global_context, server_settings[ServerSetting::dns_cache_update_period], server_settings[ServerSetting::dns_max_consecutive_failures]);
|
||||
}
|
||||
|
||||
if (dns_cache_updater)
|
||||
@ -2067,7 +2184,7 @@ try
|
||||
|
||||
/// Set current database name before loading tables and databases because
|
||||
/// system logs may copy global context.
|
||||
std::string default_database = server_settings.default_database.toString();
|
||||
std::string default_database = server_settings[ServerSetting::default_database].toString();
|
||||
global_context->setCurrentDatabaseNameInGlobalContext(default_database);
|
||||
|
||||
LOG_INFO(log, "Loading metadata from {}", path_str);
|
||||
@ -2103,7 +2220,7 @@ try
|
||||
waitLoad(TablesLoaderForegroundPoolId, system_startup_tasks);
|
||||
|
||||
/// Startup scripts can depend on the system log tables.
|
||||
if (config().has("startup_scripts") && !server_settings.prepare_system_log_tables_on_startup.changed)
|
||||
if (config().has("startup_scripts") && !server_settings[ServerSetting::prepare_system_log_tables_on_startup].changed)
|
||||
global_context->setServerSetting("prepare_system_log_tables_on_startup", true);
|
||||
|
||||
/// After attaching system databases we can initialize system log.
|
||||
@ -2123,7 +2240,7 @@ try
|
||||
database_catalog.loadMarkedAsDroppedTables();
|
||||
database_catalog.createBackgroundTasks();
|
||||
/// Then, load remaining databases (some of them maybe be loaded asynchronously)
|
||||
load_metadata_tasks = loadMetadata(global_context, default_database, server_settings.async_load_databases);
|
||||
load_metadata_tasks = loadMetadata(global_context, default_database, server_settings[ServerSetting::async_load_databases]);
|
||||
/// If we need to convert database engines, disable async tables loading
|
||||
convertDatabasesEnginesIfNeed(load_metadata_tasks, global_context);
|
||||
database_catalog.startupBackgroundTasks();
|
||||
@ -2274,11 +2391,11 @@ try
|
||||
startup_watch.stop();
|
||||
ProfileEvents::increment(ProfileEvents::ServerStartupMilliseconds, startup_watch.elapsedMilliseconds());
|
||||
|
||||
CannotAllocateThreadFaultInjector::setFaultProbability(server_settings.cannot_allocate_thread_fault_injection_probability);
|
||||
CannotAllocateThreadFaultInjector::setFaultProbability(server_settings[ServerSetting::cannot_allocate_thread_fault_injection_probability]);
|
||||
|
||||
#if USE_GWP_ASAN
|
||||
GWPAsan::initFinished();
|
||||
GWPAsan::setForceSampleProbability(server_settings.gwp_asan_force_sample_probability);
|
||||
GWPAsan::setForceSampleProbability(server_settings[ServerSetting::gwp_asan_force_sample_probability]);
|
||||
#endif
|
||||
|
||||
try
|
||||
@ -2328,15 +2445,15 @@ try
|
||||
/// Wait for unfinished backups and restores.
|
||||
/// This must be done after closing listening sockets (no more backups/restores) but before ProcessList::killAllQueries
|
||||
/// (because killAllQueries() will cancel all running backups/restores).
|
||||
if (server_settings.shutdown_wait_backups_and_restores)
|
||||
if (server_settings[ServerSetting::shutdown_wait_backups_and_restores])
|
||||
global_context->waitAllBackupsAndRestores();
|
||||
|
||||
/// Killing remaining queries.
|
||||
if (!server_settings.shutdown_wait_unfinished_queries)
|
||||
if (!server_settings[ServerSetting::shutdown_wait_unfinished_queries])
|
||||
global_context->getProcessList().killAllQueries();
|
||||
|
||||
if (current_connections)
|
||||
current_connections = waitServersToFinish(servers, servers_lock, server_settings.shutdown_wait_unfinished);
|
||||
current_connections = waitServersToFinish(servers, servers_lock, server_settings[ServerSetting::shutdown_wait_unfinished]);
|
||||
|
||||
if (current_connections)
|
||||
LOG_WARNING(log, "Closed connections. But {} remain."
|
||||
@ -2475,8 +2592,8 @@ void Server::createServers(
|
||||
|
||||
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
||||
http_params->setTimeout(settings[Setting::http_receive_timeout]);
|
||||
http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout);
|
||||
http_params->setMaxKeepAliveRequests(static_cast<int>(global_context->getServerSettings().max_keep_alive_requests));
|
||||
http_params->setKeepAliveTimeout(global_context->getServerSettings()[ServerSetting::keep_alive_timeout]);
|
||||
http_params->setMaxKeepAliveRequests(static_cast<int>(global_context->getServerSettings()[ServerSetting::max_keep_alive_requests]));
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys protocols;
|
||||
config.keys("protocols", protocols);
|
||||
@ -2732,7 +2849,7 @@ void Server::createInterserverServers(
|
||||
|
||||
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
||||
http_params->setTimeout(settings[Setting::http_receive_timeout]);
|
||||
http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout);
|
||||
http_params->setKeepAliveTimeout(global_context->getServerSettings()[ServerSetting::keep_alive_timeout]);
|
||||
|
||||
/// Now iterate over interserver_listen_hosts
|
||||
for (const auto & interserver_listen_host : interserver_listen_hosts)
|
||||
|
@ -33,6 +33,12 @@ namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsGroupArrayActionWhenLimitReached aggregate_function_group_array_action_when_limit_is_reached;
|
||||
extern const ServerSettingsUInt64 aggregate_function_group_array_max_element_size;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
@ -746,7 +752,7 @@ inline AggregateFunctionPtr createAggregateFunctionGroupArrayImpl(const DataType
|
||||
size_t getMaxArraySize()
|
||||
{
|
||||
if (auto context = Context::getGlobalContextInstance())
|
||||
return context->getServerSettings().aggregate_function_group_array_max_element_size;
|
||||
return context->getServerSettings()[ServerSetting::aggregate_function_group_array_max_element_size];
|
||||
|
||||
return 0xFFFFFF;
|
||||
}
|
||||
@ -754,7 +760,7 @@ size_t getMaxArraySize()
|
||||
bool discardOnLimitReached()
|
||||
{
|
||||
if (auto context = Context::getGlobalContextInstance())
|
||||
return context->getServerSettings().aggregate_function_group_array_action_when_limit_is_reached
|
||||
return context->getServerSettings()[ServerSetting::aggregate_function_group_array_action_when_limit_is_reached]
|
||||
== GroupArrayActionWhenLimitReached::DISCARD;
|
||||
|
||||
return false;
|
||||
|
@ -11,6 +11,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsString default_replica_name;
|
||||
extern const ServerSettingsString default_replica_path;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
void visitStorageSystemTableEngine(ASTStorage &, const DDLAdjustingForBackupVisitor::Data & data)
|
||||
@ -55,8 +61,8 @@ namespace
|
||||
zookeeper_path_arg.replace(uuid_pos, table_uuid_str.size(), "{uuid}");
|
||||
}
|
||||
const auto & server_settings = data.global_context->getServerSettings();
|
||||
if ((zookeeper_path_arg == server_settings.default_replica_path.value)
|
||||
&& (replica_name_arg == server_settings.default_replica_name.value)
|
||||
if ((zookeeper_path_arg == server_settings[ServerSetting::default_replica_path].value)
|
||||
&& (replica_name_arg == server_settings[ServerSetting::default_replica_name].value)
|
||||
&& ((engine_args.size() == 2) || !engine_args[2]->as<ASTLiteral>()))
|
||||
{
|
||||
engine_args.erase(engine_args.begin(), engine_args.begin() + 2);
|
||||
|
@ -10,6 +10,10 @@ namespace Setting
|
||||
{
|
||||
extern const SettingsSeconds http_receive_timeout;
|
||||
}
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsSeconds keep_alive_timeout;
|
||||
}
|
||||
|
||||
LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_)
|
||||
: IBridgeHelper(context_)
|
||||
@ -18,7 +22,7 @@ LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_)
|
||||
, http_timeout(context_->getGlobalContext()->getSettingsRef()[Setting::http_receive_timeout].value)
|
||||
, bridge_host(config.getString("library_bridge.host", DEFAULT_HOST))
|
||||
, bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT))
|
||||
, http_timeouts(ConnectionTimeouts::getHTTPTimeouts(context_->getSettingsRef(), context_->getServerSettings().keep_alive_timeout))
|
||||
, http_timeouts(ConnectionTimeouts::getHTTPTimeouts(context_->getSettingsRef(), context_->getServerSettings()))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -99,7 +99,7 @@ protected:
|
||||
{
|
||||
auto buf = BuilderRWBufferFromHTTP(getPingURI())
|
||||
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
|
||||
.withTimeouts(getHTTPTimeouts())
|
||||
.withTimeouts(ConnectionTimeouts::getHTTPTimeouts(getContext()->getSettingsRef(), getContext()->getServerSettings()))
|
||||
.withSettings(getContext()->getReadSettings())
|
||||
.create(credentials);
|
||||
|
||||
@ -165,11 +165,6 @@ private:
|
||||
|
||||
Poco::Net::HTTPBasicCredentials credentials{};
|
||||
|
||||
ConnectionTimeouts getHTTPTimeouts()
|
||||
{
|
||||
return ConnectionTimeouts::getHTTPTimeouts(getContext()->getSettingsRef(), getContext()->getServerSettings().keep_alive_timeout);
|
||||
}
|
||||
|
||||
protected:
|
||||
using URLParams = std::vector<std::pair<std::string, std::string>>;
|
||||
|
||||
@ -206,7 +201,7 @@ protected:
|
||||
auto buf = BuilderRWBufferFromHTTP(uri)
|
||||
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
|
||||
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
|
||||
.withTimeouts(getHTTPTimeouts())
|
||||
.withTimeouts(ConnectionTimeouts::getHTTPTimeouts(getContext()->getSettingsRef(), getContext()->getServerSettings()))
|
||||
.withSettings(getContext()->getReadSettings())
|
||||
.create(credentials);
|
||||
|
||||
@ -233,7 +228,7 @@ protected:
|
||||
auto buf = BuilderRWBufferFromHTTP(uri)
|
||||
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
|
||||
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
|
||||
.withTimeouts(getHTTPTimeouts())
|
||||
.withTimeouts(ConnectionTimeouts::getHTTPTimeouts(getContext()->getSettingsRef(), getContext()->getServerSettings()))
|
||||
.withSettings(getContext()->getReadSettings())
|
||||
.create(credentials);
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/Defines.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteIntText.h>
|
||||
|
@ -1,11 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
namespace Util
|
||||
@ -19,7 +17,7 @@ namespace DB
|
||||
struct CoordinationSettingsImpl;
|
||||
class WriteBufferFromOwnString;
|
||||
|
||||
/// List of available types supported in MergeTreeSettings object
|
||||
/// List of available types supported in CoordinationSettings object
|
||||
#define COORDINATION_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, LogsLevel) \
|
||||
|
@ -1,12 +1,204 @@
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <IO/MMappedFileCache.h>
|
||||
#include <IO/UncompressedCache.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Storages/MarkCache.h>
|
||||
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
|
||||
#include <Storages/System/ServerSettingColumnsParams.h>
|
||||
#include <Common/Config/ConfigReloader.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric BackgroundSchedulePoolSize;
|
||||
extern const Metric BackgroundBufferFlushSchedulePoolSize;
|
||||
extern const Metric BackgroundDistributedSchedulePoolSize;
|
||||
extern const Metric BackgroundMessageBrokerSchedulePoolSize;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)
|
||||
#define LIST_OF_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, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0) \
|
||||
M(UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0) \
|
||||
M(UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0) \
|
||||
M(UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0) \
|
||||
M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 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(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \
|
||||
M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \
|
||||
M(UInt64, max_unexpected_parts_loading_thread_pool_size, 8, "The number of threads to load inactive set of data parts (Unexpected ones) at startup.", 0) \
|
||||
M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \
|
||||
M(UInt64, max_mutations_bandwidth_for_server, 0, "The maximum read speed of all mutations on server in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_merges_bandwidth_for_server, 0, "The maximum read speed of all merges on server in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0) \
|
||||
M(UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0) \
|
||||
M(UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0) \
|
||||
M(UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0) \
|
||||
M(UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0) \
|
||||
M(UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0) \
|
||||
M(Bool, shutdown_wait_backups_and_restores, true, "If set to true ClickHouse will wait for running backups and restores to finish before shutdown.", 0) \
|
||||
M(Double, cannot_allocate_thread_fault_injection_probability, 0, "For testing purposes.", 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 heavy asynchronous metrics.", 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, "The maximum amount of storage that could be used for external aggregation, joins or sorting.", 0) \
|
||||
M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \
|
||||
M(UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0) \
|
||||
M(GroupArrayActionWhenLimitReached, aggregate_function_group_array_action_when_limit_is_reached, GroupArrayActionWhenLimitReached::THROW, "Action to execute when max array element size is exceeded in groupArray: `throw` exception, or `discard` extra values", 0) \
|
||||
M(UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. 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(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0) \
|
||||
M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0) \
|
||||
M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \
|
||||
M(UInt64, cgroups_memory_usage_observer_wait_time, 15, "Polling interval in seconds to read the current memory usage from cgroups. Zero means disabled.", 0) \
|
||||
M(Double, cgroup_memory_watcher_hard_limit_ratio, 0.95, "Hard memory limit ratio for cgroup memory usage observer", 0) \
|
||||
M(Double, cgroup_memory_watcher_soft_limit_ratio, 0.9, "Soft memory limit ratio limit for cgroup memory usage observer", 0) \
|
||||
M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \
|
||||
M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \
|
||||
M(Bool, ignore_empty_sql_security_in_create_view_query, true, "If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. This setting is only necessary for the migration period and will become obsolete in 24.4", 0) \
|
||||
M(UInt64, max_build_vector_similarity_index_thread_pool_size, 16, "The maximum number of threads to use to build vector similarity indexes. 0 means all cores.", 0) \
|
||||
\
|
||||
/* Database Catalog */ \
|
||||
M(UInt64, database_atomic_delay_before_drop_table_sec, 8 * 60, "The delay during which a dropped table can be restored using the UNDROP statement. If DROP TABLE ran with a SYNC modifier, the setting is ignored.", 0) \
|
||||
M(UInt64, database_catalog_unused_dir_hide_timeout_sec, 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and this directory was not modified for last database_catalog_unused_dir_hide_timeout_sec seconds, the task will 'hide' this directory by removing all access rights. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'immediately'.", 0) \
|
||||
M(UInt64, database_catalog_unused_dir_rm_timeout_sec, 30 * 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and it was previously 'hidden' (see database_catalog_unused_dir_hide_timeout_sec) and this directory was not modified for last database_catalog_unused_dir_rm_timeout_sec seconds, the task will remove this directory. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'never'.", 0) \
|
||||
M(UInt64, database_catalog_unused_dir_cleanup_period_sec, 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. Sets scheduling period of the task. Zero means 'never'.", 0) \
|
||||
M(UInt64, database_catalog_drop_error_cooldown_sec, 5, "In case if drop table failed, ClickHouse will wait for this timeout before retrying the operation.", 0) \
|
||||
M(UInt64, database_catalog_drop_table_concurrency, 16, "The size of the threadpool used for dropping tables.", 0) \
|
||||
\
|
||||
\
|
||||
M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_waiting_queries, 0, "Maximum number of concurrently waiting queries blocked due to `async_load_databases`. Note that waiting queries are not considered by `max_concurrent_*queries*` limits. Zero means unlimited.", 0) \
|
||||
\
|
||||
M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size to RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \
|
||||
M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \
|
||||
M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \
|
||||
M(Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0) \
|
||||
M(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \
|
||||
M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \
|
||||
M(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \
|
||||
M(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0) \
|
||||
M(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0) \
|
||||
M(Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0) \
|
||||
M(String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0) \
|
||||
M(UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0) \
|
||||
M(Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0) \
|
||||
M(UInt64, page_cache_chunk_size, 2 << 20, "Bytes per chunk in userspace page cache. Rounded up to a multiple of page size (typically 4 KiB) or huge page size (typically 2 MiB, only if page_cache_use_thp is enabled).", 0) \
|
||||
M(UInt64, page_cache_mmap_size, 1 << 30, "Bytes per memory mapping in userspace page cache. Not important.", 0) \
|
||||
M(UInt64, page_cache_size, 0, "Amount of virtual memory to map for userspace page cache. If page_cache_use_madv_free is enabled, it's recommended to set this higher than the machine's RAM size. Use 0 to disable userspace page cache.", 0) \
|
||||
M(Bool, page_cache_use_madv_free, DBMS_DEFAULT_PAGE_CACHE_USE_MADV_FREE, "If true, the userspace page cache will allow the OS to automatically reclaim memory from the cache on memory pressure (using MADV_FREE).", 0) \
|
||||
M(Bool, page_cache_use_transparent_huge_pages, true, "Userspace will attempt to use transparent huge pages on Linux. This is best-effort.", 0) \
|
||||
M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \
|
||||
M(UInt64, compiled_expression_cache_size, DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE, "Byte size of compiled expressions cache.", 0) \
|
||||
M(UInt64, compiled_expression_cache_elements_size, DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES, "Maximum entries in compiled expressions cache.", 0) \
|
||||
\
|
||||
M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \
|
||||
M(UInt64, dns_cache_max_entries, 10000, "Internal DNS cache max entries.", 0) \
|
||||
M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \
|
||||
M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \
|
||||
M(Bool, dns_allow_resolve_names_to_ipv4, true, "Allows resolve names to ipv4 addresses.", 0) \
|
||||
M(Bool, dns_allow_resolve_names_to_ipv6, true, "Allows resolve names to ipv6 addresses.", 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, max_table_num_to_warn, 5000lu, "If the number of tables is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_view_num_to_warn, 10000lu, "If the number of views is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_dictionary_num_to_warn, 1000lu, "If the number of dictionaries is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_database_num_to_warn, 1000lu, "If the number of databases is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_part_num_to_warn, 100000lu, "If the number of parts is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_table_num_to_throw, 0lu, "If number of tables is greater than this value, server will throw an exception. 0 means no limitation. View, remote tables, dictionary, system tables are not counted. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.", 0) \
|
||||
M(UInt64, max_database_num_to_throw, 0lu, "If number of databases is greater than this value, server will throw an exception. 0 means no limitation.", 0) \
|
||||
M(UInt64, max_authentication_methods_per_user, 100, "The maximum number of authentication methods a user can be created with or altered. Changing this setting does not affect existing users. Zero means unlimited", 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, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \
|
||||
\
|
||||
M(UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0) \
|
||||
M(Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0) \
|
||||
M(String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0) \
|
||||
M(UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0) \
|
||||
M(UInt64, background_fetches_pool_size, 16, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0) \
|
||||
M(UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0) \
|
||||
M(UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0) \
|
||||
M(UInt64, background_schedule_pool_size, 512, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
|
||||
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
|
||||
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
|
||||
M(UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0) \
|
||||
M(UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0) \
|
||||
M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \
|
||||
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \
|
||||
M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \
|
||||
M(UInt64, max_keep_alive_requests, 10000, "The maximum number of requests handled via a single http keepalive connection before the server closes this connection.", 0) \
|
||||
M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \
|
||||
M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \
|
||||
M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \
|
||||
M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \
|
||||
M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
|
||||
M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
|
||||
M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
|
||||
M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \
|
||||
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
|
||||
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
|
||||
M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \
|
||||
M(Bool, database_replicated_allow_detach_permanently, true, "Allow detaching tables permanently in Replicated databases", 0) \
|
||||
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
|
||||
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
|
||||
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \
|
||||
M(UInt64, disk_connections_soft_limit, 5000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \
|
||||
M(UInt64, disk_connections_warn_limit, 10000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the disks connections.", 0) \
|
||||
M(UInt64, disk_connections_store_limit, 30000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \
|
||||
M(UInt64, storage_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the storages connections.", 0) \
|
||||
M(UInt64, storage_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the storages connections.", 0) \
|
||||
M(UInt64, storage_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the storages connections.", 0) \
|
||||
M(UInt64, http_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
|
||||
M(UInt64, http_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
|
||||
M(UInt64, http_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
|
||||
M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \
|
||||
M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \
|
||||
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(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \
|
||||
M(Double, gwp_asan_force_sample_probability, 0.0003, "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) \
|
||||
M(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \
|
||||
M(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0) \
|
||||
M(UInt64, keeper_multiread_batch_size, 10'000, "Maximum size of batch for MultiRead request to [Zoo]Keeper that support batching. If set to 0, batching is disabled. Available only in ClickHouse Cloud.", 0) \
|
||||
M(Bool, use_legacy_mongodb_integration, true, "Use the legacy MongoDB integration implementation. Note: it's highly recommended to set this option to false, since legacy implementation will be removed in the future. Please submit any issues you encounter with the new implementation.", 0) \
|
||||
|
||||
void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
|
||||
struct ServerSettingsImpl : public BaseSettings<ServerSettingsTraits>
|
||||
{
|
||||
void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)
|
||||
|
||||
void ServerSettingsImpl::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
// settings which can be loaded from the the default profile, see also MAKE_DEPRECATED_BY_SERVER_CONFIG in src/Core/Settings.h
|
||||
std::unordered_set<std::string> settings_from_profile_allowlist = {
|
||||
@ -35,4 +227,125 @@ void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfigurat
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) ServerSettings##TYPE NAME = &ServerSettings##Impl ::NAME;
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
LIST_OF_SERVER_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS)
|
||||
}
|
||||
|
||||
#undef INITIALIZE_SETTING_EXTERN
|
||||
|
||||
ServerSettings::ServerSettings() : impl(std::make_unique<ServerSettingsImpl>())
|
||||
{
|
||||
}
|
||||
|
||||
ServerSettings::ServerSettings(const ServerSettings & settings) : impl(std::make_unique<ServerSettingsImpl>(*settings.impl))
|
||||
{
|
||||
}
|
||||
|
||||
ServerSettings::~ServerSettings() = default;
|
||||
|
||||
#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \
|
||||
const SettingField##TYPE & ServerSettings::operator[](CLASS_NAME##TYPE t) const \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
} \
|
||||
SettingField##TYPE & ServerSettings::operator[](CLASS_NAME##TYPE t) \
|
||||
{ \
|
||||
return impl.get()->*t; \
|
||||
}
|
||||
|
||||
SERVER_SETTINGS_SUPPORTED_TYPES(ServerSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR)
|
||||
#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR
|
||||
|
||||
void ServerSettings::set(std::string_view name, const Field & value)
|
||||
{
|
||||
impl->set(name, value);
|
||||
}
|
||||
|
||||
void ServerSettings::loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
impl->loadSettingsFromConfig(config);
|
||||
}
|
||||
|
||||
|
||||
void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParams & params) const
|
||||
{
|
||||
MutableColumns & res_columns = params.res_columns;
|
||||
ContextPtr context = params.context;
|
||||
|
||||
/// When the server configuration file is periodically re-loaded from disk, the server components (e.g. memory tracking) are updated
|
||||
/// with new the setting values but the settings themselves are not stored between re-loads. As a result, if one wants to know the
|
||||
/// current setting values, one needs to ask the components directly.
|
||||
std::unordered_map<String, std::pair<String, ChangeableWithoutRestart>> changeable_settings
|
||||
= {{"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_concurrent_insert_queries",
|
||||
{std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_concurrent_select_queries",
|
||||
{std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_waiting_queries", {std::to_string(context->getProcessList().getMaxWaitingQueriesAmount()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"background_buffer_flush_schedule_pool_size",
|
||||
{std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)),
|
||||
ChangeableWithoutRestart::IncreaseOnly}},
|
||||
{"background_schedule_pool_size",
|
||||
{std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}},
|
||||
{"background_message_broker_schedule_pool_size",
|
||||
{std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize)),
|
||||
ChangeableWithoutRestart::IncreaseOnly}},
|
||||
{"background_distributed_schedule_pool_size",
|
||||
{std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize)),
|
||||
ChangeableWithoutRestart::IncreaseOnly}},
|
||||
|
||||
{"mark_cache_size", {std::to_string(context->getMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"uncompressed_cache_size", {std::to_string(context->getUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"index_mark_cache_size", {std::to_string(context->getIndexMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"index_uncompressed_cache_size",
|
||||
{std::to_string(context->getIndexUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}},
|
||||
{"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}},
|
||||
{"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}}};
|
||||
|
||||
if (context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
changeable_settings.insert(
|
||||
{"background_pool_size",
|
||||
{std::to_string(context->getMergeMutateExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
changeable_settings.insert(
|
||||
{"background_move_pool_size",
|
||||
{std::to_string(context->getMovesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
changeable_settings.insert(
|
||||
{"background_fetches_pool_size",
|
||||
{std::to_string(context->getFetchesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
changeable_settings.insert(
|
||||
{"background_common_pool_size",
|
||||
{std::to_string(context->getCommonExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
}
|
||||
|
||||
for (const auto & setting : impl->all())
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
|
||||
const auto & changeable_settings_it = changeable_settings.find(setting_name);
|
||||
const bool is_changeable = (changeable_settings_it != changeable_settings.end());
|
||||
|
||||
res_columns[0]->insert(setting_name);
|
||||
res_columns[1]->insert(is_changeable ? changeable_settings_it->second.first : setting.getValueString());
|
||||
res_columns[2]->insert(setting.getDefaultValueString());
|
||||
res_columns[3]->insert(setting.isValueChanged());
|
||||
res_columns[4]->insert(setting.getDescription());
|
||||
res_columns[5]->insert(setting.getTypeName());
|
||||
res_columns[6]->insert(is_changeable ? changeable_settings_it->second.second : ChangeableWithoutRestart::No);
|
||||
res_columns[7]->insert(setting.isObsolete());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,10 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
|
||||
#include <Core/BaseSettings.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/BaseSettingsFwdMacros.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
namespace Poco::Util
|
||||
{
|
||||
@ -13,177 +11,47 @@ class AbstractConfiguration;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Field;
|
||||
struct ServerSettingColumnsParams;
|
||||
struct ServerSettingsImpl;
|
||||
|
||||
// clang-format off
|
||||
#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, shutdown_wait_unfinished, 5, "Delay in seconds to wait for unfinished queries", 0) \
|
||||
M(UInt64, max_thread_pool_size, 10000, "The maximum number of threads that could be allocated from the OS and used for query execution and background operations.", 0) \
|
||||
M(UInt64, max_thread_pool_free_size, 1000, "The maximum number of threads that will always stay in a global thread pool once allocated and remain idle in case of insufficient number of tasks.", 0) \
|
||||
M(UInt64, thread_pool_queue_size, 10000, "The maximum number of tasks that will be placed in a queue and wait for execution.", 0) \
|
||||
M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 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(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \
|
||||
M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \
|
||||
M(UInt64, max_unexpected_parts_loading_thread_pool_size, 8, "The number of threads to load inactive set of data parts (Unexpected ones) at startup.", 0) \
|
||||
M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \
|
||||
M(UInt64, max_mutations_bandwidth_for_server, 0, "The maximum read speed of all mutations on server in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_merges_bandwidth_for_server, 0, "The maximum read speed of all merges on server in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_remote_write_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_local_read_bandwidth_for_server, 0, "The maximum speed of local reads in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_local_write_bandwidth_for_server, 0, "The maximum speed of local writes in bytes per second. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0) \
|
||||
M(UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0) \
|
||||
M(UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0) \
|
||||
M(UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0) \
|
||||
M(UInt64, max_backup_bandwidth_for_server, 0, "The maximum read speed in bytes per second for all backups on server. Zero means unlimited.", 0) \
|
||||
M(UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0) \
|
||||
M(Bool, shutdown_wait_backups_and_restores, true, "If set to true ClickHouse will wait for running backups and restores to finish before shutdown.", 0) \
|
||||
M(Double, cannot_allocate_thread_fault_injection_probability, 0, "For testing purposes.", 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 heavy asynchronous metrics.", 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, "The maximum amount of storage that could be used for external aggregation, joins or sorting.", 0) \
|
||||
M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \
|
||||
M(UInt64, aggregate_function_group_array_max_element_size, 0xFFFFFF, "Max array element size in bytes for groupArray function. This limit is checked at serialization and help to avoid large state size.", 0) \
|
||||
M(GroupArrayActionWhenLimitReached, aggregate_function_group_array_action_when_limit_is_reached, GroupArrayActionWhenLimitReached::THROW, "Action to execute when max array element size is exceeded in groupArray: `throw` exception, or `discard` extra values", 0) \
|
||||
M(UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. 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(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0) \
|
||||
M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0) \
|
||||
M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \
|
||||
M(UInt64, cgroups_memory_usage_observer_wait_time, 15, "Polling interval in seconds to read the current memory usage from cgroups. Zero means disabled.", 0) \
|
||||
M(Double, cgroup_memory_watcher_hard_limit_ratio, 0.95, "Hard memory limit ratio for cgroup memory usage observer", 0) \
|
||||
M(Double, cgroup_memory_watcher_soft_limit_ratio, 0.9, "Soft memory limit ratio limit for cgroup memory usage observer", 0) \
|
||||
M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \
|
||||
M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \
|
||||
M(Bool, ignore_empty_sql_security_in_create_view_query, true, "If true, ClickHouse doesn't write defaults for empty SQL security statement in CREATE VIEW queries. This setting is only necessary for the migration period and will become obsolete in 24.4", 0) \
|
||||
M(UInt64, max_build_vector_similarity_index_thread_pool_size, 16, "The maximum number of threads to use to build vector similarity indexes. 0 means all cores.", 0) \
|
||||
\
|
||||
/* Database Catalog */ \
|
||||
M(UInt64, database_atomic_delay_before_drop_table_sec, 8 * 60, "The delay during which a dropped table can be restored using the UNDROP statement. If DROP TABLE ran with a SYNC modifier, the setting is ignored.", 0) \
|
||||
M(UInt64, database_catalog_unused_dir_hide_timeout_sec, 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and this directory was not modified for last database_catalog_unused_dir_hide_timeout_sec seconds, the task will 'hide' this directory by removing all access rights. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'immediately'.", 0) \
|
||||
M(UInt64, database_catalog_unused_dir_rm_timeout_sec, 30 * 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. If some subdirectory is not used by clickhouse-server and it was previously 'hidden' (see database_catalog_unused_dir_hide_timeout_sec) and this directory was not modified for last database_catalog_unused_dir_rm_timeout_sec seconds, the task will remove this directory. It also works for directories that clickhouse-server does not expect to see inside store/. Zero means 'never'.", 0) \
|
||||
M(UInt64, database_catalog_unused_dir_cleanup_period_sec, 24 * 60 * 60, "Parameter of a task that cleans up garbage from store/ directory. Sets scheduling period of the task. Zero means 'never'.", 0) \
|
||||
M(UInt64, database_catalog_drop_error_cooldown_sec, 5, "In case if drop table failed, ClickHouse will wait for this timeout before retrying the operation.", 0) \
|
||||
M(UInt64, database_catalog_drop_table_concurrency, 16, "The size of the threadpool used for dropping tables.", 0) \
|
||||
\
|
||||
\
|
||||
M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \
|
||||
M(UInt64, max_waiting_queries, 0, "Maximum number of concurrently waiting queries blocked due to `async_load_databases`. Note that waiting queries are not considered by `max_concurrent_*queries*` limits. Zero means unlimited.", 0) \
|
||||
\
|
||||
M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size to RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \
|
||||
M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \
|
||||
M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \
|
||||
M(Double, uncompressed_cache_size_ratio, DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the uncompressed cache relative to the cache's total size.", 0) \
|
||||
M(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \
|
||||
M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \
|
||||
M(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \
|
||||
M(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0) \
|
||||
M(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0) \
|
||||
M(Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0) \
|
||||
M(String, index_mark_cache_policy, DEFAULT_INDEX_MARK_CACHE_POLICY, "Secondary index mark cache policy name.", 0) \
|
||||
M(UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for secondary index marks. Zero means disabled.", 0) \
|
||||
M(Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0) \
|
||||
M(UInt64, page_cache_chunk_size, 2 << 20, "Bytes per chunk in userspace page cache. Rounded up to a multiple of page size (typically 4 KiB) or huge page size (typically 2 MiB, only if page_cache_use_thp is enabled).", 0) \
|
||||
M(UInt64, page_cache_mmap_size, 1 << 30, "Bytes per memory mapping in userspace page cache. Not important.", 0) \
|
||||
M(UInt64, page_cache_size, 0, "Amount of virtual memory to map for userspace page cache. If page_cache_use_madv_free is enabled, it's recommended to set this higher than the machine's RAM size. Use 0 to disable userspace page cache.", 0) \
|
||||
M(Bool, page_cache_use_madv_free, DBMS_DEFAULT_PAGE_CACHE_USE_MADV_FREE, "If true, the userspace page cache will allow the OS to automatically reclaim memory from the cache on memory pressure (using MADV_FREE).", 0) \
|
||||
M(Bool, page_cache_use_transparent_huge_pages, true, "Userspace will attempt to use transparent huge pages on Linux. This is best-effort.", 0) \
|
||||
M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \
|
||||
M(UInt64, compiled_expression_cache_size, DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE, "Byte size of compiled expressions cache.", 0) \
|
||||
M(UInt64, compiled_expression_cache_elements_size, DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES, "Maximum entries in compiled expressions cache.", 0) \
|
||||
\
|
||||
M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \
|
||||
M(UInt64, dns_cache_max_entries, 10000, "Internal DNS cache max entries.", 0) \
|
||||
M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \
|
||||
M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \
|
||||
M(Bool, dns_allow_resolve_names_to_ipv4, true, "Allows resolve names to ipv4 addresses.", 0) \
|
||||
M(Bool, dns_allow_resolve_names_to_ipv6, true, "Allows resolve names to ipv6 addresses.", 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, max_table_num_to_warn, 5000lu, "If the number of tables is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_view_num_to_warn, 10000lu, "If the number of views is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_dictionary_num_to_warn, 1000lu, "If the number of dictionaries is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_database_num_to_warn, 1000lu, "If the number of databases is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_part_num_to_warn, 100000lu, "If the number of parts is greater than this value, the server will create a warning that will displayed to user.", 0) \
|
||||
M(UInt64, max_table_num_to_throw, 0lu, "If number of tables is greater than this value, server will throw an exception. 0 means no limitation. View, remote tables, dictionary, system tables are not counted. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.", 0) \
|
||||
M(UInt64, max_database_num_to_throw, 0lu, "If number of databases is greater than this value, server will throw an exception. 0 means no limitation.", 0) \
|
||||
M(UInt64, max_authentication_methods_per_user, 100, "The maximum number of authentication methods a user can be created with or altered. Changing this setting does not affect existing users. Zero means unlimited", 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, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \
|
||||
\
|
||||
M(UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0) \
|
||||
M(Float, background_merges_mutations_concurrency_ratio, 2, "The number of part mutation tasks that can be executed concurrently by each thread in background pool.", 0) \
|
||||
M(String, background_merges_mutations_scheduling_policy, "round_robin", "The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ", 0) \
|
||||
M(UInt64, background_move_pool_size, 8, "The maximum number of threads that will be used for moving data parts to another disk or volume for *MergeTree-engine tables in a background.", 0) \
|
||||
M(UInt64, background_fetches_pool_size, 16, "The maximum number of threads that will be used for fetching data parts from another replica for *MergeTree-engine tables in a background.", 0) \
|
||||
M(UInt64, background_common_pool_size, 8, "The maximum number of threads that will be used for performing a variety of operations (mostly garbage collection) for *MergeTree-engine tables in a background.", 0) \
|
||||
M(UInt64, background_buffer_flush_schedule_pool_size, 16, "The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in a background.", 0) \
|
||||
M(UInt64, background_schedule_pool_size, 512, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
|
||||
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
|
||||
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
|
||||
M(UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0) \
|
||||
M(UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0) \
|
||||
M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \
|
||||
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \
|
||||
M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \
|
||||
M(UInt64, max_keep_alive_requests, 10000, "The maximum number of requests handled via a single http keepalive connection before the server closes this connection.", 0) \
|
||||
M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \
|
||||
M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \
|
||||
M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \
|
||||
M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \
|
||||
M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
|
||||
M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
|
||||
M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
|
||||
M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \
|
||||
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
|
||||
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
|
||||
M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \
|
||||
M(Bool, database_replicated_allow_detach_permanently, true, "Allow detaching tables permanently in Replicated databases", 0) \
|
||||
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
|
||||
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
|
||||
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \
|
||||
M(UInt64, disk_connections_soft_limit, 5000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \
|
||||
M(UInt64, disk_connections_warn_limit, 10000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the disks connections.", 0) \
|
||||
M(UInt64, disk_connections_store_limit, 30000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \
|
||||
M(UInt64, storage_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the storages connections.", 0) \
|
||||
M(UInt64, storage_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the storages connections.", 0) \
|
||||
M(UInt64, storage_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the storages connections.", 0) \
|
||||
M(UInt64, http_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
|
||||
M(UInt64, http_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
|
||||
M(UInt64, http_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
|
||||
M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \
|
||||
M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \
|
||||
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(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \
|
||||
M(Double, gwp_asan_force_sample_probability, 0.0003, "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) \
|
||||
M(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \
|
||||
M(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0) \
|
||||
M(UInt64, keeper_multiread_batch_size, 10'000, "Maximum size of batch for MultiRead request to [Zoo]Keeper that support batching. If set to 0, batching is disabled. Available only in ClickHouse Cloud.", 0) \
|
||||
M(Bool, use_legacy_mongodb_integration, true, "Use the legacy MongoDB integration implementation. Note: it's highly recommended to set this option to false, since legacy implementation will be removed in the future. Please submit any issues you encounter with the new implementation.", 0) \
|
||||
/// List of available types supported in ServerSettings object
|
||||
#define SERVER_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \
|
||||
M(CLASS_NAME, Bool) \
|
||||
M(CLASS_NAME, Double) \
|
||||
M(CLASS_NAME, GroupArrayActionWhenLimitReached) \
|
||||
M(CLASS_NAME, Float) \
|
||||
M(CLASS_NAME, Int32) \
|
||||
M(CLASS_NAME, Seconds) \
|
||||
M(CLASS_NAME, String) \
|
||||
M(CLASS_NAME, UInt32) \
|
||||
M(CLASS_NAME, UInt64)
|
||||
|
||||
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp
|
||||
SERVER_SETTINGS_SUPPORTED_TYPES(ServerSettings, DECLARE_SETTING_TRAIT)
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)
|
||||
|
||||
struct ServerSettings : public BaseSettings<ServerSettingsTraits>
|
||||
struct ServerSettings
|
||||
{
|
||||
void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
};
|
||||
enum class ChangeableWithoutRestart : uint8_t
|
||||
{
|
||||
No,
|
||||
IncreaseOnly,
|
||||
DecreaseOnly,
|
||||
Yes
|
||||
};
|
||||
|
||||
ServerSettings();
|
||||
ServerSettings(const ServerSettings & settings);
|
||||
~ServerSettings();
|
||||
|
||||
void set(std::string_view name, const Field & value);
|
||||
|
||||
void loadSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
SERVER_SETTINGS_SUPPORTED_TYPES(ServerSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR)
|
||||
|
||||
void dumpToSystemServerSettingsColumns(ServerSettingColumnsParams & res_columns) const;
|
||||
|
||||
private:
|
||||
std::unique_ptr<ServerSettingsImpl> impl;
|
||||
};
|
||||
}
|
||||
|
@ -56,6 +56,12 @@ namespace MergeTreeSetting
|
||||
extern const MergeTreeSettingsString storage_policy;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsString default_replica_name;
|
||||
extern const ServerSettingsString default_replica_path;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -91,8 +97,8 @@ static void setReplicatedEngine(ASTCreateQuery * create_query, ContextPtr contex
|
||||
|
||||
/// Get replicated engine
|
||||
const auto & server_settings = context->getServerSettings();
|
||||
String replica_path = server_settings.default_replica_path;
|
||||
String replica_name = server_settings.default_replica_name;
|
||||
String replica_path = server_settings[ServerSetting::default_replica_path];
|
||||
String replica_name = server_settings[ServerSetting::default_replica_name];
|
||||
|
||||
/// Check that replica path doesn't exist
|
||||
Macros::MacroExpansionInfo info;
|
||||
|
@ -58,6 +58,11 @@ namespace Setting
|
||||
extern const SettingsBool throw_on_unsupported_query_inside_transaction;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool database_replicated_allow_detach_permanently;
|
||||
extern const ServerSettingsUInt32 max_database_replicated_create_table_thread_pool_size;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
@ -1358,7 +1363,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
|
||||
|
||||
tables_dependencies.checkNoCyclicDependencies();
|
||||
|
||||
auto allow_concurrent_table_creation = getContext()->getServerSettings().max_database_replicated_create_table_thread_pool_size > 1;
|
||||
auto allow_concurrent_table_creation = getContext()->getServerSettings()[ServerSetting::max_database_replicated_create_table_thread_pool_size] > 1;
|
||||
auto tables_to_create_by_level = tables_dependencies.getTablesSplitByDependencyLevel();
|
||||
|
||||
ThreadPoolCallbackRunnerLocal<void> runner(getDatabaseReplicatedCreateTablesThreadPool().get(), "CreateTables");
|
||||
@ -1761,7 +1766,7 @@ void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const
|
||||
{
|
||||
waitDatabaseStarted();
|
||||
|
||||
if (!local_context->getServerSettings().database_replicated_allow_detach_permanently)
|
||||
if (!local_context->getServerSettings()[ServerSetting::database_replicated_allow_detach_permanently])
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for DETACH TABLE PERMANENTLY is disabled");
|
||||
|
||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||
|
@ -40,7 +40,7 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
, configuration(configuration_)
|
||||
, sample_block(sample_block_)
|
||||
, context(context_)
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout))
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()))
|
||||
{
|
||||
credentials.setUsername(credentials_.getUsername());
|
||||
credentials.setPassword(credentials_.getPassword());
|
||||
@ -53,7 +53,7 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
|
||||
, configuration(other.configuration)
|
||||
, sample_block(other.sample_block)
|
||||
, context(Context::createCopy(other.context))
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout))
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()))
|
||||
{
|
||||
credentials.setUsername(other.credentials.getUsername());
|
||||
credentials.setPassword(other.credentials.getPassword());
|
||||
|
@ -84,7 +84,7 @@ XDBCDictionarySource::XDBCDictionarySource(
|
||||
, load_all_query(query_builder.composeLoadAllQuery())
|
||||
, bridge_helper(bridge_)
|
||||
, bridge_url(bridge_helper->getMainURI())
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context_->getSettingsRef(), context_->getServerSettings().keep_alive_timeout))
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context_->getSettingsRef(), context_->getServerSettings()))
|
||||
{
|
||||
auto url_params = bridge_helper->getURLParams(max_block_size);
|
||||
for (const auto & [name, value] : url_params)
|
||||
|
@ -18,6 +18,11 @@ namespace Setting
|
||||
extern const SettingsSeconds http_receive_timeout;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsSeconds keep_alive_timeout;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
@ -57,7 +62,7 @@ std::unique_ptr<SeekableReadBuffer> ReadBufferFromWebServer::initialize()
|
||||
const auto & settings = context->getSettingsRef();
|
||||
const auto & server_settings = context->getServerSettings();
|
||||
|
||||
auto connection_timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, server_settings.keep_alive_timeout);
|
||||
auto connection_timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, server_settings);
|
||||
connection_timeouts.withConnectionTimeout(std::max<Poco::Timespan>(settings[Setting::http_connection_timeout], Poco::Timespan(20, 0)));
|
||||
connection_timeouts.withReceiveTimeout(std::max<Poco::Timespan>(settings[Setting::http_receive_timeout], Poco::Timespan(20, 0)));
|
||||
|
||||
|
@ -12,6 +12,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool storage_metadata_write_full_object_key;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
@ -222,7 +227,7 @@ ObjectKeyWithMetadata DiskObjectStorageMetadata::popLastObject()
|
||||
|
||||
bool DiskObjectStorageMetadata::getWriteFullObjectKeySetting()
|
||||
{
|
||||
return Context::getGlobalContextInstance()->getServerSettings().storage_metadata_write_full_object_key;
|
||||
return Context::getGlobalContextInstance()->getServerSettings()[ServerSetting::storage_metadata_write_full_object_key];
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -48,7 +48,7 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lock<std::sha
|
||||
|
||||
auto timeouts = ConnectionTimeouts::getHTTPTimeouts(
|
||||
getContext()->getSettingsRef(),
|
||||
getContext()->getServerSettings().keep_alive_timeout);
|
||||
getContext()->getServerSettings());
|
||||
|
||||
auto metadata_buf = BuilderRWBufferFromHTTP(Poco::URI(fs::path(full_url) / ".index"))
|
||||
.withConnectionGroup(HTTPConnectionGroupType::DISK)
|
||||
|
@ -22,6 +22,13 @@ namespace Setting
|
||||
extern const SettingsSeconds http_receive_timeout;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsSeconds keep_alive_timeout;
|
||||
extern const ServerSettingsSeconds replicated_fetches_http_connection_timeout;
|
||||
extern const ServerSettingsSeconds replicated_fetches_http_receive_timeout;
|
||||
extern const ServerSettingsSeconds replicated_fetches_http_send_timeout;
|
||||
}
|
||||
|
||||
Poco::Timespan ConnectionTimeouts::saturate(Poco::Timespan timespan, Poco::Timespan limit)
|
||||
{
|
||||
@ -51,29 +58,29 @@ ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings
|
||||
.withSecureConnectionTimeout(settings[Setting::connect_timeout_with_failover_secure_ms]);
|
||||
}
|
||||
|
||||
ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout)
|
||||
ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings, const ServerSettings & server_settings)
|
||||
{
|
||||
return ConnectionTimeouts()
|
||||
.withConnectionTimeout(settings[Setting::http_connection_timeout])
|
||||
.withSendTimeout(settings[Setting::http_send_timeout])
|
||||
.withReceiveTimeout(settings[Setting::http_receive_timeout])
|
||||
.withHTTPKeepAliveTimeout(http_keep_alive_timeout)
|
||||
.withHTTPKeepAliveTimeout(server_settings[ServerSetting::keep_alive_timeout])
|
||||
.withTCPKeepAliveTimeout(settings[Setting::tcp_keep_alive_timeout])
|
||||
.withHandshakeTimeout(settings[Setting::handshake_timeout_ms]);
|
||||
}
|
||||
|
||||
ConnectionTimeouts ConnectionTimeouts::getFetchPartHTTPTimeouts(const ServerSettings & server_settings, const Settings & user_settings)
|
||||
{
|
||||
auto timeouts = getHTTPTimeouts(user_settings, server_settings.keep_alive_timeout);
|
||||
auto timeouts = getHTTPTimeouts(user_settings, server_settings);
|
||||
|
||||
if (server_settings.replicated_fetches_http_connection_timeout.changed)
|
||||
timeouts.connection_timeout = server_settings.replicated_fetches_http_connection_timeout;
|
||||
if (server_settings[ServerSetting::replicated_fetches_http_connection_timeout].changed)
|
||||
timeouts.connection_timeout = server_settings[ServerSetting::replicated_fetches_http_connection_timeout];
|
||||
|
||||
if (server_settings.replicated_fetches_http_send_timeout.changed)
|
||||
timeouts.send_timeout = server_settings.replicated_fetches_http_send_timeout;
|
||||
if (server_settings[ServerSetting::replicated_fetches_http_send_timeout].changed)
|
||||
timeouts.send_timeout = server_settings[ServerSetting::replicated_fetches_http_send_timeout];
|
||||
|
||||
if (server_settings.replicated_fetches_http_receive_timeout.changed)
|
||||
timeouts.receive_timeout = server_settings.replicated_fetches_http_receive_timeout;
|
||||
if (server_settings[ServerSetting::replicated_fetches_http_receive_timeout].changed)
|
||||
timeouts.receive_timeout = server_settings[ServerSetting::replicated_fetches_http_receive_timeout];
|
||||
|
||||
return timeouts;
|
||||
}
|
||||
|
@ -56,7 +56,7 @@ struct ConnectionTimeouts
|
||||
|
||||
/// Timeouts for the case when we will try many addresses in a loop.
|
||||
static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings);
|
||||
static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout);
|
||||
static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, const ServerSettings & server_settings);
|
||||
|
||||
static ConnectionTimeouts getFetchPartHTTPTimeouts(const ServerSettings & server_settings, const Settings & user_settings);
|
||||
|
||||
|
@ -24,6 +24,11 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 max_authentication_methods_per_user;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -271,7 +276,7 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
*updated_user, query, authentication_methods, {}, default_roles_from_query, settings_from_query, grantees_from_query,
|
||||
valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods,
|
||||
implicit_no_password_allowed, no_password_allowed,
|
||||
plaintext_password_allowed, getContext()->getServerSettings().max_authentication_methods_per_user);
|
||||
plaintext_password_allowed, getContext()->getServerSettings()[ServerSetting::max_authentication_methods_per_user]);
|
||||
return updated_user;
|
||||
};
|
||||
|
||||
@ -293,7 +298,7 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
*new_user, query, authentication_methods, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{},
|
||||
valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods,
|
||||
implicit_no_password_allowed, no_password_allowed,
|
||||
plaintext_password_allowed, getContext()->getServerSettings().max_authentication_methods_per_user);
|
||||
plaintext_password_allowed, getContext()->getServerSettings()[ServerSetting::max_authentication_methods_per_user]);
|
||||
new_users.emplace_back(std::move(new_user));
|
||||
}
|
||||
|
||||
|
@ -247,6 +247,33 @@ namespace MergeTreeSetting
|
||||
extern const MergeTreeSettingsString mutation_workload;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 background_buffer_flush_schedule_pool_size;
|
||||
extern const ServerSettingsUInt64 background_common_pool_size;
|
||||
extern const ServerSettingsUInt64 background_distributed_schedule_pool_size;
|
||||
extern const ServerSettingsUInt64 background_fetches_pool_size;
|
||||
extern const ServerSettingsFloat background_merges_mutations_concurrency_ratio;
|
||||
extern const ServerSettingsString background_merges_mutations_scheduling_policy;
|
||||
extern const ServerSettingsUInt64 background_message_broker_schedule_pool_size;
|
||||
extern const ServerSettingsUInt64 background_move_pool_size;
|
||||
extern const ServerSettingsUInt64 background_pool_size;
|
||||
extern const ServerSettingsUInt64 background_schedule_pool_size;
|
||||
extern const ServerSettingsBool display_secrets_in_show_and_select;
|
||||
extern const ServerSettingsUInt64 max_backup_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_build_vector_similarity_index_thread_pool_size;
|
||||
extern const ServerSettingsUInt64 max_local_read_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_local_write_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_merges_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_mutations_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_remote_read_network_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_remote_write_network_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_replicated_fetches_network_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 max_replicated_sends_network_bandwidth_for_server;
|
||||
extern const ServerSettingsUInt64 tables_loader_background_pool_size;
|
||||
extern const ServerSettingsUInt64 tables_loader_foreground_pool_size;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -857,31 +884,31 @@ struct ContextSharedPart : boost::noncopyable
|
||||
|
||||
void configureServerWideThrottling()
|
||||
{
|
||||
if (auto bandwidth = server_settings.max_replicated_fetches_network_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_replicated_fetches_network_bandwidth_for_server])
|
||||
replicated_fetches_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_replicated_sends_network_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_replicated_sends_network_bandwidth_for_server])
|
||||
replicated_sends_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_remote_read_network_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_remote_read_network_bandwidth_for_server])
|
||||
remote_read_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_remote_write_network_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_remote_write_network_bandwidth_for_server])
|
||||
remote_write_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_local_read_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_local_read_bandwidth_for_server])
|
||||
local_read_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_local_write_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_local_write_bandwidth_for_server])
|
||||
local_write_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_backup_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_backup_bandwidth_for_server])
|
||||
backups_server_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_mutations_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_mutations_bandwidth_for_server])
|
||||
mutations_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
|
||||
if (auto bandwidth = server_settings.max_merges_bandwidth_for_server)
|
||||
if (auto bandwidth = server_settings[ServerSetting::max_merges_bandwidth_for_server])
|
||||
merges_throttler = std::make_shared<Throttler>(bandwidth);
|
||||
}
|
||||
};
|
||||
@ -2350,7 +2377,7 @@ StoragePtr Context::getViewSource() const
|
||||
|
||||
bool Context::displaySecretsInShowAndSelect() const
|
||||
{
|
||||
return shared->server_settings.display_secrets_in_show_and_select;
|
||||
return shared->server_settings[ServerSetting::display_secrets_in_show_and_select];
|
||||
}
|
||||
|
||||
Settings Context::getSettingsCopy() const
|
||||
@ -2803,7 +2830,7 @@ AsyncLoader & Context::getAsyncLoader() const
|
||||
CurrentMetrics::TablesLoaderForegroundThreads,
|
||||
CurrentMetrics::TablesLoaderForegroundThreadsActive,
|
||||
CurrentMetrics::TablesLoaderForegroundThreadsScheduled,
|
||||
shared->server_settings.tables_loader_foreground_pool_size,
|
||||
shared->server_settings[ServerSetting::tables_loader_foreground_pool_size],
|
||||
TablesLoaderForegroundPriority
|
||||
},
|
||||
{ // TablesLoaderBackgroundLoadPoolId
|
||||
@ -2811,7 +2838,7 @@ AsyncLoader & Context::getAsyncLoader() const
|
||||
CurrentMetrics::TablesLoaderBackgroundThreads,
|
||||
CurrentMetrics::TablesLoaderBackgroundThreadsActive,
|
||||
CurrentMetrics::TablesLoaderBackgroundThreadsScheduled,
|
||||
shared->server_settings.tables_loader_background_pool_size,
|
||||
shared->server_settings[ServerSetting::tables_loader_background_pool_size],
|
||||
TablesLoaderBackgroundLoadPriority
|
||||
},
|
||||
{ // TablesLoaderBackgroundStartupPoolId
|
||||
@ -2819,7 +2846,7 @@ AsyncLoader & Context::getAsyncLoader() const
|
||||
CurrentMetrics::TablesLoaderBackgroundThreads,
|
||||
CurrentMetrics::TablesLoaderBackgroundThreadsActive,
|
||||
CurrentMetrics::TablesLoaderBackgroundThreadsScheduled,
|
||||
shared->server_settings.tables_loader_background_pool_size,
|
||||
shared->server_settings[ServerSetting::tables_loader_background_pool_size],
|
||||
TablesLoaderBackgroundStartupPriority
|
||||
}
|
||||
},
|
||||
@ -3387,8 +3414,8 @@ ThreadPool & Context::getBuildVectorSimilarityIndexThreadPool() const
|
||||
shared->build_vector_similarity_index_threadpool_initialized,
|
||||
[&]
|
||||
{
|
||||
size_t pool_size = shared->server_settings.max_build_vector_similarity_index_thread_pool_size > 0
|
||||
? shared->server_settings.max_build_vector_similarity_index_thread_pool_size
|
||||
size_t pool_size = shared->server_settings[ServerSetting::max_build_vector_similarity_index_thread_pool_size] > 0
|
||||
? shared->server_settings[ServerSetting::max_build_vector_similarity_index_thread_pool_size]
|
||||
: getNumberOfCPUCoresToUse();
|
||||
shared->build_vector_similarity_index_threadpool = std::make_unique<ThreadPool>(
|
||||
CurrentMetrics::BuildVectorSimilarityIndexThreads,
|
||||
@ -3403,7 +3430,7 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool() const
|
||||
{
|
||||
callOnce(shared->buffer_flush_schedule_pool_initialized, [&] {
|
||||
shared->buffer_flush_schedule_pool = std::make_unique<BackgroundSchedulePool>(
|
||||
shared->server_settings.background_buffer_flush_schedule_pool_size,
|
||||
shared->server_settings[ServerSetting::background_buffer_flush_schedule_pool_size],
|
||||
CurrentMetrics::BackgroundBufferFlushSchedulePoolTask,
|
||||
CurrentMetrics::BackgroundBufferFlushSchedulePoolSize,
|
||||
"BgBufSchPool");
|
||||
@ -3447,7 +3474,7 @@ BackgroundSchedulePool & Context::getSchedulePool() const
|
||||
{
|
||||
callOnce(shared->schedule_pool_initialized, [&] {
|
||||
shared->schedule_pool = std::make_unique<BackgroundSchedulePool>(
|
||||
shared->server_settings.background_schedule_pool_size,
|
||||
shared->server_settings[ServerSetting::background_schedule_pool_size],
|
||||
CurrentMetrics::BackgroundSchedulePoolTask,
|
||||
CurrentMetrics::BackgroundSchedulePoolSize,
|
||||
"BgSchPool");
|
||||
@ -3460,7 +3487,7 @@ BackgroundSchedulePool & Context::getDistributedSchedulePool() const
|
||||
{
|
||||
callOnce(shared->distributed_schedule_pool_initialized, [&] {
|
||||
shared->distributed_schedule_pool = std::make_unique<BackgroundSchedulePool>(
|
||||
shared->server_settings.background_distributed_schedule_pool_size,
|
||||
shared->server_settings[ServerSetting::background_distributed_schedule_pool_size],
|
||||
CurrentMetrics::BackgroundDistributedSchedulePoolTask,
|
||||
CurrentMetrics::BackgroundDistributedSchedulePoolSize,
|
||||
"BgDistSchPool");
|
||||
@ -3473,7 +3500,7 @@ BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const
|
||||
{
|
||||
callOnce(shared->message_broker_schedule_pool_initialized, [&] {
|
||||
shared->message_broker_schedule_pool = std::make_unique<BackgroundSchedulePool>(
|
||||
shared->server_settings.background_message_broker_schedule_pool_size,
|
||||
shared->server_settings[ServerSetting::background_message_broker_schedule_pool_size],
|
||||
CurrentMetrics::BackgroundMessageBrokerSchedulePoolTask,
|
||||
CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize,
|
||||
"BgMBSchPool");
|
||||
@ -5523,13 +5550,13 @@ void Context::initializeBackgroundExecutorsIfNeeded()
|
||||
return;
|
||||
|
||||
const ServerSettings & server_settings = shared->server_settings;
|
||||
size_t background_pool_size = server_settings.background_pool_size;
|
||||
auto background_merges_mutations_concurrency_ratio = server_settings.background_merges_mutations_concurrency_ratio;
|
||||
size_t background_pool_size = server_settings[ServerSetting::background_pool_size];
|
||||
auto background_merges_mutations_concurrency_ratio = server_settings[ServerSetting::background_merges_mutations_concurrency_ratio];
|
||||
size_t background_pool_max_tasks_count = static_cast<size_t>(background_pool_size * background_merges_mutations_concurrency_ratio);
|
||||
String background_merges_mutations_scheduling_policy = server_settings.background_merges_mutations_scheduling_policy;
|
||||
size_t background_move_pool_size = server_settings.background_move_pool_size;
|
||||
size_t background_fetches_pool_size = server_settings.background_fetches_pool_size;
|
||||
size_t background_common_pool_size = server_settings.background_common_pool_size;
|
||||
String background_merges_mutations_scheduling_policy = server_settings[ServerSetting::background_merges_mutations_scheduling_policy];
|
||||
size_t background_move_pool_size = server_settings[ServerSetting::background_move_pool_size];
|
||||
size_t background_fetches_pool_size = server_settings[ServerSetting::background_fetches_pool_size];
|
||||
size_t background_common_pool_size = server_settings[ServerSetting::background_common_pool_size];
|
||||
|
||||
/// With this executor we can execute more tasks than threads we have
|
||||
shared->merge_mutate_executor = std::make_shared<MergeMutateBackgroundExecutor>
|
||||
|
@ -55,6 +55,15 @@ namespace CurrentMetrics
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 database_atomic_delay_before_drop_table_sec;
|
||||
extern const ServerSettingsUInt64 database_catalog_drop_error_cooldown_sec;
|
||||
extern const ServerSettingsUInt64 database_catalog_unused_dir_cleanup_period_sec;
|
||||
extern const ServerSettingsUInt64 database_catalog_unused_dir_hide_timeout_sec;
|
||||
extern const ServerSettingsUInt64 database_catalog_unused_dir_rm_timeout_sec;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_DATABASE;
|
||||
@ -195,7 +204,7 @@ void DatabaseCatalog::initializeAndLoadTemporaryDatabase()
|
||||
void DatabaseCatalog::createBackgroundTasks()
|
||||
{
|
||||
/// It has to be done before databases are loaded (to avoid a race condition on initialization)
|
||||
if (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER && getContext()->getServerSettings().database_catalog_unused_dir_cleanup_period_sec)
|
||||
if (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER && getContext()->getServerSettings()[ServerSetting::database_catalog_unused_dir_cleanup_period_sec])
|
||||
{
|
||||
auto cleanup_task_holder
|
||||
= getContext()->getSchedulePool().createTask("DatabaseCatalogCleanupStoreDirectoryTask", [this]() { this->cleanupStoreDirectoryTask(); });
|
||||
@ -216,7 +225,7 @@ void DatabaseCatalog::startupBackgroundTasks()
|
||||
{
|
||||
(*cleanup_task)->activate();
|
||||
/// Do not start task immediately on server startup, it's not urgent.
|
||||
(*cleanup_task)->scheduleAfter(static_cast<time_t>(getContext()->getServerSettings().database_catalog_unused_dir_hide_timeout_sec) * 1000);
|
||||
(*cleanup_task)->scheduleAfter(static_cast<time_t>(getContext()->getServerSettings()[ServerSetting::database_catalog_unused_dir_hide_timeout_sec]) * 1000);
|
||||
}
|
||||
|
||||
(*drop_task)->activate();
|
||||
@ -1135,7 +1144,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
|
||||
table_id,
|
||||
table,
|
||||
dropped_metadata_path,
|
||||
drop_time + static_cast<time_t>(getContext()->getServerSettings().database_atomic_delay_before_drop_table_sec)
|
||||
drop_time + static_cast<time_t>(getContext()->getServerSettings()[ServerSetting::database_atomic_delay_before_drop_table_sec])
|
||||
});
|
||||
if (first_async_drop_in_queue == tables_marked_dropped.end())
|
||||
--first_async_drop_in_queue;
|
||||
@ -1327,7 +1336,7 @@ void DatabaseCatalog::dropTablesParallel(std::vector<DatabaseCatalog::TablesMark
|
||||
++first_async_drop_in_queue;
|
||||
|
||||
tables_marked_dropped.splice(tables_marked_dropped.end(), tables_marked_dropped, table_iterator);
|
||||
table_iterator->drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + getContext()->getServerSettings().database_catalog_drop_error_cooldown_sec;
|
||||
table_iterator->drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + getContext()->getServerSettings()[ServerSetting::database_catalog_drop_error_cooldown_sec];
|
||||
|
||||
if (first_async_drop_in_queue == tables_marked_dropped.end())
|
||||
--first_async_drop_in_queue;
|
||||
@ -1709,7 +1718,7 @@ void DatabaseCatalog::cleanupStoreDirectoryTask()
|
||||
LOG_TEST(log, "Nothing to clean up from store/ on disk {}", disk_name);
|
||||
}
|
||||
|
||||
(*cleanup_task)->scheduleAfter(static_cast<time_t>(getContext()->getServerSettings().database_catalog_unused_dir_cleanup_period_sec) * 1000);
|
||||
(*cleanup_task)->scheduleAfter(static_cast<time_t>(getContext()->getServerSettings()[ServerSetting::database_catalog_unused_dir_cleanup_period_sec]) * 1000);
|
||||
}
|
||||
|
||||
bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir)
|
||||
@ -1733,7 +1742,7 @@ bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskP
|
||||
time_t current_time = time(nullptr);
|
||||
if (st.st_mode & (S_IRWXU | S_IRWXG | S_IRWXO))
|
||||
{
|
||||
if (current_time <= max_modification_time + static_cast<time_t>(getContext()->getServerSettings().database_catalog_unused_dir_hide_timeout_sec))
|
||||
if (current_time <= max_modification_time + static_cast<time_t>(getContext()->getServerSettings()[ServerSetting::database_catalog_unused_dir_hide_timeout_sec]))
|
||||
return false;
|
||||
|
||||
LOG_INFO(log, "Removing access rights for unused directory {} from disk {} (will remove it when timeout exceed)", unused_dir, disk_name);
|
||||
@ -1748,7 +1757,7 @@ bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskP
|
||||
return true;
|
||||
}
|
||||
|
||||
auto unused_dir_rm_timeout_sec = static_cast<time_t>(getContext()->getServerSettings().database_catalog_unused_dir_rm_timeout_sec);
|
||||
auto unused_dir_rm_timeout_sec = static_cast<time_t>(getContext()->getServerSettings()[ServerSetting::database_catalog_unused_dir_rm_timeout_sec]);
|
||||
|
||||
if (!unused_dir_rm_timeout_sec)
|
||||
return false;
|
||||
|
@ -44,6 +44,11 @@ namespace Setting
|
||||
extern const SettingsSeconds lock_acquire_timeout;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool disable_insertion_and_mutation;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -200,7 +205,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
||||
|
||||
if (mutation_commands.hasNonEmptyMutationCommands() || !partition_commands.empty())
|
||||
{
|
||||
if (getContext()->getServerSettings().disable_insertion_and_mutation)
|
||||
if (getContext()->getServerSettings()[ServerSetting::disable_insertion_and_mutation])
|
||||
throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Mutations are prohibited");
|
||||
}
|
||||
|
||||
|
@ -142,6 +142,13 @@ namespace Setting
|
||||
extern const SettingsBool restore_replace_external_dictionary_source_to_null;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool ignore_empty_sql_security_in_create_view_query;
|
||||
extern const ServerSettingsUInt64 max_database_num_to_throw;
|
||||
extern const ServerSettingsUInt64 max_table_num_to_throw;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
@ -191,7 +198,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Database {} already exists.", database_name);
|
||||
}
|
||||
|
||||
auto db_num_limit = getContext()->getGlobalContext()->getServerSettings().max_database_num_to_throw;
|
||||
auto db_num_limit = getContext()->getGlobalContext()->getServerSettings()[ServerSetting::max_database_num_to_throw];
|
||||
if (db_num_limit > 0 && !internal)
|
||||
{
|
||||
size_t db_count = DatabaseCatalog::instance().getDatabases().size();
|
||||
@ -1462,7 +1469,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery();
|
||||
auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup);
|
||||
|
||||
if (!create.sql_security && create.supportSQLSecurity() && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query)
|
||||
if (!create.sql_security && create.supportSQLSecurity() && !getContext()->getServerSettings()[ServerSetting::ignore_empty_sql_security_in_create_view_query])
|
||||
create.sql_security = std::make_shared<ASTSQLSecurity>();
|
||||
|
||||
if (create.sql_security)
|
||||
@ -1907,7 +1914,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
}
|
||||
}
|
||||
|
||||
UInt64 table_num_limit = getContext()->getGlobalContext()->getServerSettings().max_table_num_to_throw;
|
||||
UInt64 table_num_limit = getContext()->getGlobalContext()->getServerSettings()[ServerSetting::max_table_num_to_throw];
|
||||
if (table_num_limit > 0 && !internal)
|
||||
{
|
||||
UInt64 table_count = CurrentMetrics::get(CurrentMetrics::AttachedTable);
|
||||
|
@ -34,6 +34,11 @@ namespace MergeTreeSetting
|
||||
extern const MergeTreeSettingsLightweightMutationProjectionMode lightweight_mutation_projection_mode;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool disable_insertion_and_mutation;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_IS_READ_ONLY;
|
||||
@ -64,7 +69,7 @@ BlockIO InterpreterDeleteQuery::execute()
|
||||
if (table->isStaticStorage())
|
||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");
|
||||
|
||||
if (getContext()->getGlobalContext()->getServerSettings().disable_insertion_and_mutation)
|
||||
if (getContext()->getGlobalContext()->getServerSettings()[ServerSetting::disable_insertion_and_mutation])
|
||||
throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Delete queries are prohibited");
|
||||
|
||||
DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
|
||||
|
@ -74,6 +74,11 @@ namespace Setting
|
||||
extern const SettingsBool enable_parsing_to_custom_serialization;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool disable_insertion_and_mutation;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
@ -757,7 +762,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
const Settings & settings = getContext()->getSettingsRef();
|
||||
auto & query = query_ptr->as<ASTInsertQuery &>();
|
||||
|
||||
if (getContext()->getServerSettings().disable_insertion_and_mutation
|
||||
if (getContext()->getServerSettings()[ServerSetting::disable_insertion_and_mutation]
|
||||
&& query.table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE)
|
||||
throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Insert queries are prohibited");
|
||||
|
||||
|
@ -188,6 +188,10 @@ namespace Setting
|
||||
extern const SettingsBool use_with_fill_by_sorting_prefix;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 max_entries_for_hash_table_stats;
|
||||
}
|
||||
|
||||
static UInt64 getLimitUIntValue(const ASTPtr & node, const ContextPtr & context, const std::string & expr);
|
||||
|
||||
@ -2743,7 +2747,7 @@ static Aggregator::Params getAggregatorParams(
|
||||
const auto stats_collecting_params = StatsCollectingParams(
|
||||
calculateCacheKey(query_ptr),
|
||||
settings[Setting::collect_hash_table_stats_during_aggregation],
|
||||
context.getServerSettings().max_entries_for_hash_table_stats,
|
||||
context.getServerSettings()[ServerSetting::max_entries_for_hash_table_stats],
|
||||
settings[Setting::max_size_to_preallocate_for_aggregation]);
|
||||
|
||||
return Aggregator::Params
|
||||
|
@ -98,6 +98,11 @@ namespace Setting
|
||||
extern const SettingsSeconds receive_timeout;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsDouble cannot_allocate_thread_fault_injection_probability;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -755,7 +760,7 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
case Type::START_THREAD_FUZZER:
|
||||
getContext()->checkAccess(AccessType::SYSTEM_THREAD_FUZZER);
|
||||
ThreadFuzzer::start();
|
||||
CannotAllocateThreadFaultInjector::setFaultProbability(getContext()->getServerSettings().cannot_allocate_thread_fault_injection_probability);
|
||||
CannotAllocateThreadFaultInjector::setFaultProbability(getContext()->getServerSettings()[ServerSetting::cannot_allocate_thread_fault_injection_probability]);
|
||||
break;
|
||||
case Type::UNFREEZE:
|
||||
{
|
||||
|
@ -54,6 +54,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool prepare_system_log_tables_on_startup;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -295,7 +300,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
|
||||
if (session_log)
|
||||
global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring.");
|
||||
|
||||
bool should_prepare = global_context->getServerSettings().prepare_system_log_tables_on_startup;
|
||||
bool should_prepare = global_context->getServerSettings()[ServerSetting::prepare_system_log_tables_on_startup];
|
||||
try
|
||||
{
|
||||
for (auto & log : getAllLogs())
|
||||
|
@ -136,6 +136,10 @@ namespace Setting
|
||||
extern const SettingsBool use_with_fill_by_sorting_prefix;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 max_entries_for_hash_table_stats;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
@ -422,7 +426,7 @@ Aggregator::Params getAggregatorParams(const PlannerContextPtr & planner_context
|
||||
const auto stats_collecting_params = StatsCollectingParams(
|
||||
calculateCacheKey(select_query_info.query),
|
||||
settings[Setting::collect_hash_table_stats_during_aggregation],
|
||||
query_context->getServerSettings().max_entries_for_hash_table_stats,
|
||||
query_context->getServerSettings()[ServerSetting::max_entries_for_hash_table_stats],
|
||||
settings[Setting::max_size_to_preallocate_for_aggregation]);
|
||||
|
||||
auto aggregate_descriptions = aggregation_analysis_result.aggregate_descriptions;
|
||||
|
@ -55,6 +55,11 @@ namespace Setting
|
||||
extern const SettingsMaxThreads max_threads;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 max_entries_for_hash_table_stats;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCOMPATIBLE_TYPE_OF_JOIN;
|
||||
@ -821,7 +826,7 @@ static std::shared_ptr<IJoin> tryCreateJoin(JoinAlgorithm algorithm,
|
||||
StatsCollectingParams params{
|
||||
calculateCacheKey(table_join, right_table_expression),
|
||||
settings[Setting::collect_hash_table_stats_during_joins],
|
||||
query_context->getServerSettings().max_entries_for_hash_table_stats,
|
||||
query_context->getServerSettings()[ServerSetting::max_entries_for_hash_table_stats],
|
||||
settings[Setting::max_size_to_preallocate_for_joins]};
|
||||
return std::make_shared<ConcurrentHashJoin>(
|
||||
query_context, table_join, query_context->getSettingsRef()[Setting::max_threads], right_table_expression_header, params);
|
||||
|
@ -6,6 +6,7 @@
|
||||
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -101,6 +101,11 @@ namespace Setting
|
||||
extern const SettingsBool wait_for_async_insert;
|
||||
extern const SettingsSeconds wait_for_async_insert_timeout;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsBool validate_tcp_client_information;
|
||||
}
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -1899,7 +1904,7 @@ void TCPHandler::receiveQuery()
|
||||
|
||||
correctQueryClientInfo(session->getClientInfo(), client_info);
|
||||
const auto & config_ref = Context::getGlobalContextInstance()->getServerSettings();
|
||||
if (config_ref.validate_tcp_client_information)
|
||||
if (config_ref[ServerSetting::validate_tcp_client_information])
|
||||
validateClientInfo(session->getClientInfo(), client_info);
|
||||
}
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/re2.h>
|
||||
|
||||
|
@ -32,6 +32,12 @@ namespace Setting
|
||||
extern const SettingsSeconds lock_acquire_timeout;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsString default_replica_name;
|
||||
extern const ServerSettingsString default_replica_path;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -62,8 +68,8 @@ RefreshTask::RefreshTask(
|
||||
const auto macros = context->getMacros();
|
||||
Macros::MacroExpansionInfo info;
|
||||
info.table_id = view->getStorageID();
|
||||
coordination.path = macros->expand(server_settings.default_replica_path, info);
|
||||
coordination.replica_name = context->getMacros()->expand(server_settings.default_replica_name, info);
|
||||
coordination.path = macros->expand(server_settings[ServerSetting::default_replica_path], info);
|
||||
coordination.replica_name = context->getMacros()->expand(server_settings[ServerSetting::default_replica_name], info);
|
||||
|
||||
auto zookeeper = context->getZooKeeper();
|
||||
String replica_path = coordination.path + "/replicas/" + coordination.replica_name;
|
||||
|
@ -29,6 +29,11 @@ namespace ProfileEvents
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 max_build_vector_similarity_index_thread_pool_size;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int FORMAT_VERSION_TOO_OLD;
|
||||
@ -270,7 +275,7 @@ void updateImpl(const ColumnArray * column_array, const ColumnArray::Offsets & c
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length");
|
||||
|
||||
/// Reserving space is mandatory
|
||||
size_t max_thread_pool_size = Context::getGlobalContextInstance()->getServerSettings().max_build_vector_similarity_index_thread_pool_size;
|
||||
size_t max_thread_pool_size = Context::getGlobalContextInstance()->getServerSettings()[ServerSetting::max_build_vector_similarity_index_thread_pool_size];
|
||||
if (max_thread_pool_size == 0)
|
||||
max_thread_pool_size = getNumberOfCPUCoresToUse();
|
||||
unum::usearch::index_limits_t limits(roundUpToPowerOfTwoOrZero(index->size() + rows), max_thread_pool_size);
|
||||
|
@ -46,6 +46,12 @@ namespace MergeTreeSetting
|
||||
extern const MergeTreeSettingsUInt64 index_granularity;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsString default_replica_name;
|
||||
extern const ServerSettingsString default_replica_path;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -259,8 +265,8 @@ static TableZnodeInfo extractZooKeeperPathAndReplicaNameFromEngineArgs(
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("registerStorageMergeTree"), "Replacing user-provided ZooKeeper path and replica name ({}, {}) "
|
||||
"with default arguments", ast_zk_path->value.safeGet<String>(), ast_replica_name->value.safeGet<String>());
|
||||
ast_zk_path->value = server_settings.default_replica_path;
|
||||
ast_replica_name->value = server_settings.default_replica_name;
|
||||
ast_zk_path->value = server_settings[ServerSetting::default_replica_path];
|
||||
ast_replica_name->value = server_settings[ServerSetting::default_replica_name];
|
||||
}
|
||||
|
||||
return expand_macro(ast_zk_path, ast_replica_name, ast_zk_path->value.safeGet<String>(), ast_replica_name->value.safeGet<String>());
|
||||
@ -283,7 +289,7 @@ static TableZnodeInfo extractZooKeeperPathAndReplicaNameFromEngineArgs(
|
||||
auto * ast_zk_path = path_arg.get();
|
||||
auto * ast_replica_name = name_arg.get();
|
||||
|
||||
auto res = expand_macro(ast_zk_path, ast_replica_name, server_settings.default_replica_path, server_settings.default_replica_name);
|
||||
auto res = expand_macro(ast_zk_path, ast_replica_name, server_settings[ServerSetting::default_replica_path], server_settings[ServerSetting::default_replica_name]);
|
||||
|
||||
engine_args.emplace_back(std::move(path_arg));
|
||||
engine_args.emplace_back(std::move(name_arg));
|
||||
|
@ -48,6 +48,11 @@ namespace Setting
|
||||
extern const SettingsSeconds lock_acquire_timeout;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt64 max_materialized_views_count_for_table;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -135,7 +140,7 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
{
|
||||
auto select_table_dependent_views = DatabaseCatalog::instance().getDependentViews(select.select_table_id);
|
||||
|
||||
auto max_materialized_views_count_for_table = getContext()->getServerSettings().max_materialized_views_count_for_table;
|
||||
auto max_materialized_views_count_for_table = getContext()->getServerSettings()[ServerSetting::max_materialized_views_count_for_table];
|
||||
if (max_materialized_views_count_for_table && select_table_dependent_views.size() >= max_materialized_views_count_for_table)
|
||||
throw Exception(ErrorCodes::TOO_MANY_MATERIALIZED_VIEWS,
|
||||
"Too many materialized views, maximum: {}", max_materialized_views_count_for_table);
|
||||
|
@ -328,7 +328,7 @@ String StorageReplicatedMergeTree::getEndpointName() const
|
||||
|
||||
static ConnectionTimeouts getHTTPTimeouts(ContextPtr context)
|
||||
{
|
||||
return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout);
|
||||
return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings());
|
||||
}
|
||||
|
||||
static MergeTreePartInfo makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(const String & partition_id)
|
||||
|
@ -132,7 +132,7 @@ String getSampleURI(String uri, ContextPtr context)
|
||||
|
||||
static ConnectionTimeouts getHTTPTimeouts(ContextPtr context)
|
||||
{
|
||||
return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings().keep_alive_timeout);
|
||||
return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings());
|
||||
}
|
||||
|
||||
IStorageURLBase::IStorageURLBase(
|
||||
|
@ -25,6 +25,11 @@ namespace Setting
|
||||
extern const SettingsBool odbc_bridge_use_connection_pooling;
|
||||
}
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsSeconds keep_alive_timeout;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
@ -151,7 +156,7 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMet
|
||||
local_context,
|
||||
ConnectionTimeouts::getHTTPTimeouts(
|
||||
local_context->getSettingsRef(),
|
||||
local_context->getServerSettings().keep_alive_timeout),
|
||||
local_context->getServerSettings()),
|
||||
compression_method);
|
||||
}
|
||||
|
||||
|
18
src/Storages/System/ServerSettingColumnsParams.h
Normal file
18
src/Storages/System/ServerSettingColumnsParams.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ServerSettingColumnsParams
|
||||
{
|
||||
MutableColumns & res_columns;
|
||||
ContextPtr context;
|
||||
|
||||
ServerSettingColumnsParams(MutableColumns & res_columns_, ContextPtr context_)
|
||||
: res_columns(res_columns_), context(context_)
|
||||
{
|
||||
}
|
||||
};
|
||||
}
|
@ -1,46 +1,24 @@
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#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>
|
||||
#include <Storages/System/ServerSettingColumnsParams.h>
|
||||
#include <Storages/System/StorageSystemServerSettings.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric BackgroundSchedulePoolSize;
|
||||
extern const Metric BackgroundBufferFlushSchedulePoolSize;
|
||||
extern const Metric BackgroundDistributedSchedulePoolSize;
|
||||
extern const Metric BackgroundMessageBrokerSchedulePoolSize;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
enum class ChangeableWithoutRestart : uint8_t
|
||||
{
|
||||
No,
|
||||
IncreaseOnly,
|
||||
DecreaseOnly,
|
||||
Yes
|
||||
};
|
||||
|
||||
ColumnsDescription StorageSystemServerSettings::getColumnsDescription()
|
||||
{
|
||||
auto changeable_without_restart_type = std::make_shared<DataTypeEnum8>(
|
||||
DataTypeEnum8::Values
|
||||
{
|
||||
{"No", static_cast<Int8>(ChangeableWithoutRestart::No)},
|
||||
{"IncreaseOnly", static_cast<Int8>(ChangeableWithoutRestart::IncreaseOnly)},
|
||||
{"DecreaseOnly", static_cast<Int8>(ChangeableWithoutRestart::DecreaseOnly)},
|
||||
{"Yes", static_cast<Int8>(ChangeableWithoutRestart::Yes)},
|
||||
{"No", static_cast<Int8>(ServerSettings::ChangeableWithoutRestart::No)},
|
||||
{"IncreaseOnly", static_cast<Int8>(ServerSettings::ChangeableWithoutRestart::IncreaseOnly)},
|
||||
{"DecreaseOnly", static_cast<Int8>(ServerSettings::ChangeableWithoutRestart::DecreaseOnly)},
|
||||
{"Yes", static_cast<Int8>(ServerSettings::ChangeableWithoutRestart::Yes)},
|
||||
});
|
||||
|
||||
return ColumnsDescription
|
||||
@ -58,64 +36,12 @@ ColumnsDescription StorageSystemServerSettings::getColumnsDescription()
|
||||
|
||||
void StorageSystemServerSettings::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) const
|
||||
{
|
||||
/// When the server configuration file is periodically re-loaded from disk, the server components (e.g. memory tracking) are updated
|
||||
/// with new the setting values but the settings themselves are not stored between re-loads. As a result, if one wants to know the
|
||||
/// current setting values, one needs to ask the components directly.
|
||||
std::unordered_map<String, std::pair<String, ChangeableWithoutRestart>> changeable_settings = {
|
||||
{"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_concurrent_insert_queries", {std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_concurrent_select_queries", {std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_waiting_queries", {std::to_string(context->getProcessList().getMaxWaitingQueriesAmount()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"background_buffer_flush_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}},
|
||||
{"background_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}},
|
||||
{"background_message_broker_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}},
|
||||
{"background_distributed_schedule_pool_size", {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}},
|
||||
|
||||
{"mark_cache_size", {std::to_string(context->getMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"uncompressed_cache_size", {std::to_string(context->getUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"index_mark_cache_size", {std::to_string(context->getIndexMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"index_uncompressed_cache_size", {std::to_string(context->getIndexUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
{"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}},
|
||||
|
||||
{"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}},
|
||||
{"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}},
|
||||
{"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}}
|
||||
};
|
||||
|
||||
if (context->areBackgroundExecutorsInitialized())
|
||||
{
|
||||
changeable_settings.insert({"background_pool_size", {std::to_string(context->getMergeMutateExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
changeable_settings.insert({"background_move_pool_size", {std::to_string(context->getMovesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
changeable_settings.insert({"background_fetches_pool_size", {std::to_string(context->getFetchesExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
changeable_settings.insert({"background_common_pool_size", {std::to_string(context->getCommonExecutor()->getMaxThreads()), ChangeableWithoutRestart::IncreaseOnly}});
|
||||
}
|
||||
|
||||
const auto & config = context->getConfigRef();
|
||||
ServerSettings settings;
|
||||
settings.loadSettingsFromConfig(config);
|
||||
|
||||
for (const auto & setting : settings.all())
|
||||
{
|
||||
const auto & setting_name = setting.getName();
|
||||
|
||||
const auto & changeable_settings_it = changeable_settings.find(setting_name);
|
||||
const bool is_changeable = (changeable_settings_it != changeable_settings.end());
|
||||
|
||||
res_columns[0]->insert(setting_name);
|
||||
res_columns[1]->insert(is_changeable ? changeable_settings_it->second.first : setting.getValueString());
|
||||
res_columns[2]->insert(setting.getDefaultValueString());
|
||||
res_columns[3]->insert(setting.isValueChanged());
|
||||
res_columns[4]->insert(setting.getDescription());
|
||||
res_columns[5]->insert(setting.getTypeName());
|
||||
res_columns[6]->insert(is_changeable ? changeable_settings_it->second.second : ChangeableWithoutRestart::No);
|
||||
res_columns[7]->insert(setting.isObsolete());
|
||||
}
|
||||
ServerSettingColumnsParams params{res_columns, context};
|
||||
settings.dumpToSystemServerSettingsColumns(params);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -171,7 +171,7 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex
|
||||
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
|
||||
.withTimeouts(ConnectionTimeouts::getHTTPTimeouts(
|
||||
context->getSettingsRef(),
|
||||
context->getServerSettings().keep_alive_timeout))
|
||||
context->getServerSettings()))
|
||||
.create(credentials);
|
||||
|
||||
std::string columns_info;
|
||||
|
@ -56,19 +56,21 @@ find -L $ROOT_PATH -type l 2>/dev/null | grep -v contrib && echo "^ Broken symli
|
||||
SETTINGS_FILE=$(mktemp)
|
||||
ALL_DECLARATION_FILES="
|
||||
$ROOT_PATH/src/Core/Settings.cpp
|
||||
$ROOT_PATH/src/Core/ServerSettings.cpp
|
||||
$ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp
|
||||
$ROOT_PATH/src/Coordination/CoordinationSettings.cpp
|
||||
$ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h"
|
||||
|
||||
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep "M(" | awk '{print substr($2, 0, length($2) - 1) " Settings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq > ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp | grep "M(" | awk '{print substr($2, 0, length($2) - 1) " MergeTreeSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Coordination/CoordinationSettings.cpp | grep "M(" | awk '{print substr($2, 0, length($2) - 1) " CoordinationSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
|
||||
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " Settings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq > ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Core/ServerSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " ServerSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Storages/MergeTree/MergeTreeSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " MergeTreeSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
cat $ROOT_PATH/src/Coordination/CoordinationSettings.cpp | grep " M(" | awk '{print substr($2, 0, length($2) - 1) " CoordinationSettings" substr($1, 3, length($1) - 3) " SettingsDeclaration" }' | sort | uniq >> ${SETTINGS_FILE}
|
||||
|
||||
# Check that if there are duplicated settings (declared in different objects) they all have the same type (it's simpler to validate style with that assert)
|
||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
sed -e 's/CoordinationSettings//g' \
|
||||
-e 's/MergeTreeSettings//g' \
|
||||
-e 's/ServerSettings//g' \
|
||||
-e 's/Settings//g' | \
|
||||
sort | uniq | awk '{ print $1 }' | uniq -d);
|
||||
do
|
||||
@ -79,6 +81,7 @@ done
|
||||
# We append all uses of extern found in implementation files to validate them in a single pass and avoid reading the same files over and over
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | \
|
||||
xargs grep -e "^\s*extern const Settings" \
|
||||
-e "^\s**extern const ServerSettings" \
|
||||
-e "^\s**extern const MergeTreeSettings" \
|
||||
-e "^\s**extern const CoordinationSettings" -T | \
|
||||
awk '{print substr($5, 0, length($5) -1) " " $4 " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
|
||||
@ -100,11 +103,11 @@ done
|
||||
# Look for settings declared with multiple types
|
||||
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | \
|
||||
sed -e 's/MergeTreeSettings//g' \
|
||||
-e 's/ServerSettings//g' \
|
||||
-e 's/CoordinationSettings//g' \
|
||||
-e 's/Settings//g' | \
|
||||
sort | uniq | awk '{ print $1 }' | sort | uniq -d);
|
||||
do
|
||||
echo $setting
|
||||
expected=$(grep "^$setting " ${SETTINGS_FILE} | grep SettingsDeclaration | awk '{ print $2 }')
|
||||
grep "^$setting " ${SETTINGS_FILE} | grep -v " $expected" | awk '{ print $3 " found setting " $1 " with type " $2 }' | while read line;
|
||||
do
|
||||
|
Loading…
Reference in New Issue
Block a user