Implement ServerSetting 'allowed_feature_tier'

This commit is contained in:
Raúl Marín 2024-11-12 16:56:01 +01:00
parent 356393b488
commit 6ad72f1b8f
8 changed files with 112 additions and 28 deletions

View File

@ -166,6 +166,7 @@ namespace MergeTreeSetting
namespace ServerSetting
{
extern const ServerSettingsUInt32 allowed_feature_tier;
extern const ServerSettingsUInt32 asynchronous_heavy_metrics_update_period_s;
extern const ServerSettingsUInt32 asynchronous_metrics_update_period_s;
extern const ServerSettingsBool asynchronous_metrics_enable_heavy_metrics;
@ -1771,6 +1772,7 @@ try
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]);
global_context->getAccessControl().setAllowTierSettings(new_server_settings[ServerSetting::allowed_feature_tier]);
/// Only for system.server_settings
global_context->setConfigReloaderInterval(new_server_settings[ServerSetting::config_reload_interval_ms]);

View File

@ -876,4 +876,28 @@ void AccessControl::allowAllSettings()
custom_settings_prefixes->registerPrefixes({""});
}
void AccessControl::setAllowTierSettings(UInt32 value)
{
allow_experimental_tier_settings = value == 0;
allow_beta_tier_settings = value <= 1;
}
UInt32 AccessControl::getAllowTierSettings() const
{
if (allow_experimental_tier_settings)
return 0;
if (allow_beta_tier_settings)
return 1;
return 2;
}
bool AccessControl::getAllowExperimentalTierSettings() const
{
return allow_experimental_tier_settings;
}
bool AccessControl::getAllowBetaTierSettings() const
{
return allow_beta_tier_settings;
}
}

View File

@ -243,6 +243,11 @@ public:
/// Allow all setting names - this can be used in clients to pass-through unknown settings to the server.
void allowAllSettings();
void setAllowTierSettings(UInt32 value);
UInt32 getAllowTierSettings() const;
bool getAllowExperimentalTierSettings() const;
bool getAllowBetaTierSettings() const;
private:
class ContextAccessCache;
class CustomSettingsPrefixes;
@ -272,6 +277,8 @@ private:
std::atomic_bool table_engines_require_grant = false;
std::atomic_int bcrypt_workfactor = 12;
std::atomic<AuthenticationType> default_password_type = AuthenticationType::SHA256_PASSWORD;
std::atomic_bool allow_experimental_tier_settings = true;
std::atomic_bool allow_beta_tier_settings = true;
};
}

View File

@ -404,6 +404,28 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu
if (current_settings[Setting::readonly] > 1 && resolved_name == "readonly")
return Checker(PreformattedMessage::create("Cannot modify 'readonly' setting in readonly mode"), ErrorCodes::READONLY);
if (access_control)
{
bool allowed_experimental = access_control->getAllowExperimentalTierSettings();
bool allowed_beta = access_control->getAllowBetaTierSettings();
if (!allowed_experimental || !allowed_beta)
{
auto setting_tier = current_settings.getTier(resolved_name);
if (setting_tier == SettingsTierType::EXPERIMENTAL && !allowed_experimental)
return Checker(
PreformattedMessage::create(
"Cannot modify setting '{}'. Changes to EXPERIMENTAL settings are disabled in the server config ('allowed_feature_tier')",
setting_name),
ErrorCodes::READONLY);
if (setting_tier == SettingsTierType::BETA && !allowed_beta)
return Checker(
PreformattedMessage::create(
"Cannot modify setting '{}'. Changes to BETA settings are disabled in the server config ('allowed_feature_tier')",
setting_name),
ErrorCodes::READONLY);
}
}
auto it = constraints.find(resolved_name);
if (current_settings[Setting::readonly] == 1)
{

View File

@ -131,6 +131,7 @@ public:
const char * getTypeName(std::string_view name) const;
const char * getDescription(std::string_view name) const;
SettingsTierType getTier(std::string_view name) const;
/// Checks if it's possible to assign a field to a specified value and throws an exception if not.
/// This function doesn't change the fields, it performs check only.
@ -380,6 +381,18 @@ const char * BaseSettings<TTraits>::getDescription(std::string_view name) const
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename TTraits>
SettingsTierType BaseSettings<TTraits>::getTier(std::string_view name) const
{
name = TTraits::resolveName(name);
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.getTier(index);
if (tryGetCustomSetting(name))
return SettingsTierType::PRODUCTION;
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename TTraits>
void BaseSettings<TTraits>::checkCanSet(std::string_view name, const Field & value)
{

View File

@ -1,3 +1,4 @@
#include <Access/AccessControl.h>
#include <Core/BackgroundSchedulePool.h>
#include <Core/BaseSettings.h>
#include <Core/BaseSettingsFwdMacrosImpl.h>
@ -26,6 +27,8 @@ extern const Metric BackgroundMessageBrokerSchedulePoolSize;
namespace DB
{
// clang-format off
#define LIST_OF_SERVER_SETTINGS(DECLARE, ALIAS) \
DECLARE(Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0) \
DECLARE(Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0) \
@ -198,7 +201,11 @@ namespace DB
DECLARE(UInt64, load_marks_threadpool_pool_size, 50, "Size of background pool for marks loading", 0) \
DECLARE(UInt64, load_marks_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \
DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \
DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0)
DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \
DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \
// clang-format on
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below
@ -281,7 +288,8 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam
/// 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_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}},
@ -294,16 +302,13 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam
{"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}},
{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}},
{std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}},
{"background_distributed_schedule_pool_size",
{std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize)),
ChangeableWithoutRestart::IncreaseOnly}},
{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}},
@ -314,7 +319,11 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam
{"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}},
{"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}},
{"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}}};
{"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}},
{"allowed_feature_tier",
{std::to_string(context->getAccessControl().getAllowTierSettings()), ChangeableWithoutRestart::Yes}},
};
if (context->areBackgroundExecutorsInitialized())
{

View File

@ -6176,6 +6176,11 @@ bool Settings::isChanged(std::string_view name) const
return impl->isChanged(name);
}
SettingsTierType Settings::getTier(std::string_view name) const
{
return impl->getTier(name);
}
bool Settings::tryGet(std::string_view name, Field & value) const
{
return impl->tryGet(name, value);

View File

@ -4,6 +4,7 @@
#include <Core/Field.h>
#include <Core/SettingsEnums.h>
#include <Core/SettingsFields.h>
#include <Core/SettingsTierType.h>
#include <Core/SettingsWriteFormat.h>
#include <base/types.h>
#include <Common/SettingsChanges.h>
@ -117,6 +118,7 @@ struct Settings
/// General API as needed
bool has(std::string_view name) const;
bool isChanged(std::string_view name) const;
SettingsTierType getTier(std::string_view name) const;
bool tryGet(std::string_view name, Field & value) const;
Field get(std::string_view name) const;