2017-09-19 20:42:42 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeSettings.h>
|
2019-04-18 23:29:32 +00:00
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
2023-02-04 18:41:41 +00:00
|
|
|
#include <Disks/getOrCreateDiskFromAST.h>
|
2017-09-19 20:42:42 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2019-03-23 22:45:28 +00:00
|
|
|
#include <Parsers/ASTSetQuery.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
2023-02-04 18:31:19 +00:00
|
|
|
#include <Parsers/FieldFromAST.h>
|
2023-02-22 11:14:59 +00:00
|
|
|
#include <Parsers/isDiskFunction.h>
|
2023-02-04 18:31:19 +00:00
|
|
|
#include <Core/Field.h>
|
2017-09-19 20:42:42 +00:00
|
|
|
#include <Common/Exception.h>
|
2023-02-04 18:31:19 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2020-12-10 22:05:02 +00:00
|
|
|
#include <Core/Settings.h>
|
2017-09-19 20:42:42 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2019-04-18 23:29:32 +00:00
|
|
|
extern const int UNKNOWN_SETTING;
|
2020-10-26 04:39:20 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2017-09-19 20:42:42 +00:00
|
|
|
}
|
|
|
|
|
2020-07-20 09:57:17 +00:00
|
|
|
IMPLEMENT_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS)
|
2019-04-18 23:29:32 +00:00
|
|
|
|
2018-07-08 04:54:37 +00:00
|
|
|
void MergeTreeSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
|
2017-09-19 20:42:42 +00:00
|
|
|
{
|
2017-09-20 14:41:07 +00:00
|
|
|
if (!config.has(config_elem))
|
|
|
|
return;
|
|
|
|
|
|
|
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
|
|
|
config.keys(config_elem, config_keys);
|
|
|
|
|
2019-04-18 23:29:32 +00:00
|
|
|
try
|
2017-09-20 14:41:07 +00:00
|
|
|
{
|
2019-04-18 23:29:32 +00:00
|
|
|
for (const String & key : config_keys)
|
|
|
|
set(key, config.getString(config_elem + "." + key));
|
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
|
|
|
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
|
2020-10-26 03:35:57 +00:00
|
|
|
e.addMessage("in MergeTree config");
|
|
|
|
throw;
|
2017-09-20 14:41:07 +00:00
|
|
|
}
|
2017-09-19 20:42:42 +00:00
|
|
|
}
|
|
|
|
|
2024-01-16 13:25:09 +00:00
|
|
|
void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr context, bool is_attach)
|
2017-09-19 20:42:42 +00:00
|
|
|
{
|
|
|
|
if (storage_def.settings)
|
|
|
|
{
|
2019-04-18 23:29:32 +00:00
|
|
|
try
|
2017-09-19 20:42:42 +00:00
|
|
|
{
|
2023-02-17 15:54:04 +00:00
|
|
|
bool found_disk_setting = false;
|
|
|
|
bool found_storage_policy_setting = false;
|
|
|
|
|
2023-02-04 18:31:19 +00:00
|
|
|
auto changes = storage_def.settings->changes;
|
2023-02-06 16:20:29 +00:00
|
|
|
for (auto & [name, value] : changes)
|
2023-02-04 18:31:19 +00:00
|
|
|
{
|
2023-02-06 16:20:29 +00:00
|
|
|
CustomType custom;
|
2023-02-17 15:54:04 +00:00
|
|
|
if (name == "disk")
|
2023-02-04 18:31:19 +00:00
|
|
|
{
|
2023-02-17 15:54:04 +00:00
|
|
|
if (value.tryGet<CustomType>(custom) && 0 == strcmp(custom.getTypeName(), "AST"))
|
2023-02-04 18:31:19 +00:00
|
|
|
{
|
2023-02-17 15:54:04 +00:00
|
|
|
auto ast = dynamic_cast<const FieldFromASTImpl &>(custom.getImpl()).ast;
|
|
|
|
if (ast && isDiskFunction(ast))
|
|
|
|
{
|
2024-01-16 13:25:09 +00:00
|
|
|
auto disk_name = getOrCreateDiskFromDiskAST(ast, context, is_attach);
|
2024-01-23 17:04:50 +00:00
|
|
|
LOG_TRACE(getLogger("MergeTreeSettings"), "Created custom disk {}", disk_name);
|
2023-02-17 15:54:04 +00:00
|
|
|
value = disk_name;
|
|
|
|
}
|
2023-02-04 18:31:19 +00:00
|
|
|
}
|
2023-02-17 15:54:04 +00:00
|
|
|
|
|
|
|
if (has("storage_policy"))
|
|
|
|
resetToDefault("storage_policy");
|
|
|
|
|
|
|
|
found_disk_setting = true;
|
2023-02-04 18:31:19 +00:00
|
|
|
}
|
2023-02-17 15:54:04 +00:00
|
|
|
else if (name == "storage_policy")
|
|
|
|
found_storage_policy_setting = true;
|
|
|
|
|
|
|
|
if (found_disk_setting && found_storage_policy_setting)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"MergeTree settings `storage_policy` and `disk` cannot be specified at the same time");
|
|
|
|
}
|
|
|
|
|
2023-02-04 18:31:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
applyChanges(changes);
|
2019-04-18 23:29:32 +00:00
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
|
|
|
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
|
2020-10-26 03:35:57 +00:00
|
|
|
e.addMessage("for storage " + storage_def.engine->name);
|
|
|
|
throw;
|
2017-09-19 20:42:42 +00:00
|
|
|
}
|
|
|
|
}
|
2017-09-20 14:41:07 +00:00
|
|
|
else
|
2017-09-19 20:42:42 +00:00
|
|
|
{
|
2017-09-20 14:41:07 +00:00
|
|
|
auto settings_ast = std::make_shared<ASTSetQuery>();
|
|
|
|
settings_ast->is_standalone = false;
|
|
|
|
storage_def.set(storage_def.settings, settings_ast);
|
2017-09-19 20:42:42 +00:00
|
|
|
}
|
2017-09-20 14:41:07 +00:00
|
|
|
|
2019-04-18 23:29:32 +00:00
|
|
|
SettingsChanges & changes = storage_def.settings->changes;
|
2017-09-20 14:41:07 +00:00
|
|
|
|
|
|
|
#define ADD_IF_ABSENT(NAME) \
|
|
|
|
if (std::find_if(changes.begin(), changes.end(), \
|
2023-02-03 20:03:13 +00:00
|
|
|
[](const SettingChange & c) { return c.name == #NAME; }) \
|
2017-09-20 14:41:07 +00:00
|
|
|
== changes.end()) \
|
2020-03-18 02:02:24 +00:00
|
|
|
changes.push_back(SettingChange{#NAME, (NAME).value});
|
2017-09-20 14:41:07 +00:00
|
|
|
|
2019-01-04 14:18:49 +00:00
|
|
|
APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(ADD_IF_ABSENT)
|
2017-09-20 14:41:07 +00:00
|
|
|
#undef ADD_IF_ABSENT
|
2017-09-19 20:42:42 +00:00
|
|
|
}
|
|
|
|
|
2022-05-26 12:14:58 +00:00
|
|
|
void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const
|
2020-07-30 19:08:13 +00:00
|
|
|
{
|
2022-05-26 12:14:58 +00:00
|
|
|
if (number_of_free_entries_in_pool_to_execute_mutation > background_pool_tasks)
|
2020-07-30 19:08:13 +00:00
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_mutation' setting"
|
|
|
|
" ({}) (default values are defined in <merge_tree> section of config.xml"
|
|
|
|
" or the value can be specified per table in SETTINGS section of CREATE TABLE query)"
|
2021-09-30 21:26:24 +00:00
|
|
|
" is greater than the value of 'background_pool_size'*'background_merges_mutations_concurrency_ratio'"
|
2020-07-30 19:08:13 +00:00
|
|
|
" ({}) (the value is defined in users.xml for default profile)."
|
|
|
|
" This indicates incorrect configuration because mutations cannot work with these settings.",
|
|
|
|
number_of_free_entries_in_pool_to_execute_mutation,
|
2022-05-26 12:14:58 +00:00
|
|
|
background_pool_tasks);
|
2020-07-30 19:08:13 +00:00
|
|
|
}
|
|
|
|
|
2022-05-26 12:14:58 +00:00
|
|
|
if (number_of_free_entries_in_pool_to_lower_max_size_of_merge > background_pool_tasks)
|
2020-07-30 19:08:13 +00:00
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting"
|
|
|
|
" ({}) (default values are defined in <merge_tree> section of config.xml"
|
|
|
|
" or the value can be specified per table in SETTINGS section of CREATE TABLE query)"
|
2021-09-30 21:26:24 +00:00
|
|
|
" is greater than the value of 'background_pool_size'*'background_merges_mutations_concurrency_ratio'"
|
2020-07-30 19:08:13 +00:00
|
|
|
" ({}) (the value is defined in users.xml for default profile)."
|
|
|
|
" This indicates incorrect configuration because the maximum size of merge will be always lowered.",
|
2020-08-13 15:41:26 +00:00
|
|
|
number_of_free_entries_in_pool_to_lower_max_size_of_merge,
|
2022-05-26 12:14:58 +00:00
|
|
|
background_pool_tasks);
|
2020-07-30 19:08:13 +00:00
|
|
|
}
|
|
|
|
|
2023-08-14 10:17:06 +00:00
|
|
|
if (number_of_free_entries_in_pool_to_execute_optimize_entire_partition > background_pool_tasks)
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_optimize_entire_partition' setting"
|
|
|
|
" ({}) (default values are defined in <merge_tree> section of config.xml"
|
|
|
|
" or the value can be specified per table in SETTINGS section of CREATE TABLE query)"
|
|
|
|
" is greater than the value of 'background_pool_size'*'background_merges_mutations_concurrency_ratio'"
|
|
|
|
" ({}) (the value is defined in users.xml for default profile)."
|
|
|
|
" This indicates incorrect configuration because the maximum size of merge will be always lowered.",
|
|
|
|
number_of_free_entries_in_pool_to_execute_optimize_entire_partition,
|
|
|
|
background_pool_tasks);
|
|
|
|
}
|
|
|
|
|
2022-12-25 15:06:34 +00:00
|
|
|
// Zero index_granularity is nonsensical.
|
|
|
|
if (index_granularity < 1)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"index_granularity: value {} makes no sense",
|
|
|
|
index_granularity);
|
|
|
|
}
|
|
|
|
|
2021-03-08 09:38:07 +00:00
|
|
|
// The min_index_granularity_bytes value is 1024 b and index_granularity_bytes is 10 mb by default.
|
|
|
|
// If index_granularity_bytes is not disabled i.e > 0 b, then always ensure that it's greater than
|
|
|
|
// min_index_granularity_bytes. This is mainly a safeguard against accidents whereby a really low
|
|
|
|
// index_granularity_bytes SETTING of 1b can create really large parts with large marks.
|
|
|
|
if (index_granularity_bytes > 0 && index_granularity_bytes < min_index_granularity_bytes)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"index_granularity_bytes: {} is lower than specified min_index_granularity_bytes: {}",
|
|
|
|
index_granularity_bytes,
|
|
|
|
min_index_granularity_bytes);
|
|
|
|
}
|
|
|
|
|
|
|
|
// If min_bytes_to_rebalance_partition_over_jbod is not disabled i.e > 0 b, then always ensure that
|
|
|
|
// it's not less than min_bytes_to_rebalance_partition_over_jbod. This is a safeguard to avoid tiny
|
|
|
|
// parts to participate JBOD balancer which will slow down the merge process.
|
|
|
|
if (min_bytes_to_rebalance_partition_over_jbod > 0
|
|
|
|
&& min_bytes_to_rebalance_partition_over_jbod < max_bytes_to_merge_at_max_space_in_pool / 1024)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS,
|
2022-03-10 10:29:42 +00:00
|
|
|
"min_bytes_to_rebalance_partition_over_jbod: {} is lower than specified max_bytes_to_merge_at_max_space_in_pool / 1024: {}",
|
2021-03-08 09:38:07 +00:00
|
|
|
min_bytes_to_rebalance_partition_over_jbod,
|
|
|
|
max_bytes_to_merge_at_max_space_in_pool / 1024);
|
|
|
|
}
|
2023-05-22 23:25:17 +00:00
|
|
|
|
|
|
|
if (max_cleanup_delay_period < cleanup_delay_period)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"The value of max_cleanup_delay_period setting ({}) must be greater than the value of cleanup_delay_period setting ({})",
|
|
|
|
max_cleanup_delay_period, cleanup_delay_period);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (max_merge_selecting_sleep_ms < merge_selecting_sleep_ms)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"The value of max_merge_selecting_sleep_ms setting ({}) must be greater than the value of merge_selecting_sleep_ms setting ({})",
|
|
|
|
max_merge_selecting_sleep_ms, merge_selecting_sleep_ms);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (merge_selecting_sleep_slowdown_factor < 1.f)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"The value of merge_selecting_sleep_slowdown_factor setting ({}) cannot be less than 1.0",
|
|
|
|
merge_selecting_sleep_slowdown_factor);
|
|
|
|
}
|
2021-03-08 09:38:07 +00:00
|
|
|
}
|
2023-12-23 03:55:55 +00:00
|
|
|
|
2023-09-29 04:25:34 +00:00
|
|
|
void MergeTreeColumnSettings::validate(const SettingsChanges & changes)
|
|
|
|
{
|
2023-10-02 05:55:39 +00:00
|
|
|
static const MergeTreeSettings merge_tree_settings;
|
|
|
|
static const std::set<String> allowed_column_level_settings =
|
|
|
|
{
|
|
|
|
"min_compress_block_size",
|
|
|
|
"max_compress_block_size"
|
|
|
|
};
|
2024-01-18 19:52:13 +00:00
|
|
|
|
2023-09-29 04:25:34 +00:00
|
|
|
for (const auto & change : changes)
|
|
|
|
{
|
|
|
|
if (!allowed_column_level_settings.contains(change.name))
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::UNKNOWN_SETTING,
|
|
|
|
"Setting {} is unknown or not supported at column level, supported settings: {}",
|
|
|
|
change.name,
|
|
|
|
fmt::join(allowed_column_level_settings, ", "));
|
|
|
|
merge_tree_settings.checkCanSet(change.name, change.value);
|
|
|
|
}
|
|
|
|
}
|
2023-10-02 05:55:39 +00:00
|
|
|
|
2023-12-23 03:55:55 +00:00
|
|
|
|
|
|
|
std::vector<String> MergeTreeSettings::getAllRegisteredNames() const
|
|
|
|
{
|
|
|
|
std::vector<String> all_settings;
|
|
|
|
for (const auto & setting_field : all())
|
|
|
|
all_settings.push_back(setting_field.getName());
|
|
|
|
return all_settings;
|
|
|
|
}
|
|
|
|
|
2017-09-19 20:42:42 +00:00
|
|
|
}
|