mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge pull request #71841 from Algunenano/experimental_tiers_switch
Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features
This commit is contained in:
commit
3ae055c74e
@ -3286,3 +3286,17 @@ Use the legacy MongoDB integration implementation. Deprecated.
|
||||
Type: Bool
|
||||
|
||||
Default value: `true`.
|
||||
|
||||
## allowed_feature_tier
|
||||
|
||||
Controls if the user can change settings related to the different feature tiers.
|
||||
0 - Changes to any setting are allowed (experimental, beta, production).
|
||||
1 - Only changes to beta and production feature settings are allowed. Changes to experimental settings are rejected.
|
||||
2 - Only changes to production settings are allowed. Changes to experimental or beta settings are rejected.
|
||||
|
||||
This is equivalent to setting a readonly constraint on all EXPERIMENTAL / BETA features.
|
||||
```
|
||||
|
||||
Type: UInt32
|
||||
|
||||
Default value: `0` (all settings can be changed).
|
||||
|
@ -79,6 +79,7 @@ namespace Setting
|
||||
|
||||
namespace ServerSetting
|
||||
{
|
||||
extern const ServerSettingsUInt32 allowed_feature_tier;
|
||||
extern const ServerSettingsDouble cache_size_to_ram_max_ratio;
|
||||
extern const ServerSettingsUInt64 compiled_expression_cache_elements_size;
|
||||
extern const ServerSettingsUInt64 compiled_expression_cache_size;
|
||||
@ -789,6 +790,9 @@ void LocalServer::processConfig()
|
||||
/// Initialize a dummy query cache.
|
||||
global_context->setQueryCache(0, 0, 0, 0);
|
||||
|
||||
/// Initialize allowed tiers
|
||||
global_context->getAccessControl().setAllowTierSettings(server_settings[ServerSetting::allowed_feature_tier]);
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
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];
|
||||
|
@ -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]);
|
||||
|
||||
@ -2161,9 +2163,12 @@ try
|
||||
|
||||
/// Check sanity of MergeTreeSettings on server startup
|
||||
{
|
||||
/// All settings can be changed in the global config
|
||||
bool allowed_experimental = true;
|
||||
bool allowed_beta = true;
|
||||
size_t background_pool_tasks = global_context->getMergeMutateExecutor()->getMaxTasksCount();
|
||||
global_context->getMergeTreeSettings().sanityCheck(background_pool_tasks);
|
||||
global_context->getReplicatedMergeTreeSettings().sanityCheck(background_pool_tasks);
|
||||
global_context->getMergeTreeSettings().sanityCheck(background_pool_tasks, allowed_experimental, allowed_beta);
|
||||
global_context->getReplicatedMergeTreeSettings().sanityCheck(background_pool_tasks, allowed_experimental, allowed_beta);
|
||||
}
|
||||
/// try set up encryption. There are some errors in config, error will be printed and server wouldn't start.
|
||||
CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs");
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -397,13 +397,35 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu
|
||||
|
||||
/** The `readonly` value is understood as follows:
|
||||
* 0 - no read-only restrictions.
|
||||
* 1 - only read requests, as well as changing settings with `changable_in_readonly` flag.
|
||||
* 1 - only read requests, as well as changing settings with `changeable_in_readonly` flag.
|
||||
* 2 - only read requests, as well as changing settings, except for the `readonly` setting.
|
||||
*/
|
||||
|
||||
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)
|
||||
{
|
||||
|
@ -40,7 +40,7 @@ class AccessControl;
|
||||
* <const/>
|
||||
* </force_index_by_date>
|
||||
* <max_threads>
|
||||
* <changable_in_readonly/>
|
||||
* <changeable_in_readonly/>
|
||||
* </max_threads>
|
||||
* </constraints>
|
||||
* </user_profile>
|
||||
@ -50,7 +50,7 @@ class AccessControl;
|
||||
* If a setting cannot be change due to the read-only mode this class throws an exception.
|
||||
* The value of `readonly` is understood as follows:
|
||||
* 0 - not read-only mode, no additional checks.
|
||||
* 1 - only read queries, as well as changing settings with <changable_in_readonly/> flag.
|
||||
* 1 - only read queries, as well as changing settings with <changeable_in_readonly/> flag.
|
||||
* 2 - only read queries and you can change the settings, except for the `readonly` setting.
|
||||
*
|
||||
*/
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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) \
|
||||
@ -201,7 +204,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
|
||||
|
||||
@ -284,40 +291,42 @@ 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}},
|
||||
{"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",
|
||||
{"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",
|
||||
{"max_concurrent_select_queries",
|
||||
{std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), ChangeableWithoutRestart::Yes}},
|
||||
{"max_waiting_queries", {std::to_string(context->getProcessList().getMaxWaitingQueriesAmount()), 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}},
|
||||
{"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}},
|
||||
{"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}}};
|
||||
{"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}},
|
||||
{"mutation_workload", {context->getMutationWorkload(), 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())
|
||||
{
|
||||
|
@ -4323,7 +4323,7 @@ Disable limit on kafka_num_consumers that depends on the number of available CPU
|
||||
)", 0) \
|
||||
DECLARE(Bool, allow_experimental_kafka_offsets_storage_in_keeper, false, R"(
|
||||
Allow experimental feature to store Kafka related offsets in ClickHouse Keeper. When enabled a ClickHouse Keeper path and replica name can be specified to the Kafka table engine. As a result instead of the regular Kafka engine, a new type of storage engine will be used that stores the committed offsets primarily in ClickHouse Keeper
|
||||
)", 0) \
|
||||
)", EXPERIMENTAL) \
|
||||
DECLARE(Bool, enable_software_prefetch_in_aggregation, true, R"(
|
||||
Enable use of software prefetch in aggregation
|
||||
)", 0) \
|
||||
@ -5656,10 +5656,10 @@ Build local plan for local replica
|
||||
\
|
||||
DECLARE(Bool, allow_experimental_analyzer, true, R"(
|
||||
Allow new query analyzer.
|
||||
)", IMPORTANT | BETA) ALIAS(enable_analyzer) \
|
||||
)", IMPORTANT) ALIAS(enable_analyzer) \
|
||||
DECLARE(Bool, analyzer_compatibility_join_using_top_level_identifier, false, R"(
|
||||
Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).
|
||||
)", BETA) \
|
||||
)", 0) \
|
||||
\
|
||||
DECLARE(Timezone, session_timezone, "", R"(
|
||||
Sets the implicit time zone of the current session or query.
|
||||
@ -5770,7 +5770,7 @@ Possible values:
|
||||
|
||||
- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled.
|
||||
- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled.
|
||||
)", 0) \
|
||||
)", EXPERIMENTAL) \
|
||||
DECLARE(Bool, allow_experimental_vector_similarity_index, false, R"(
|
||||
Allow experimental vector similarity index
|
||||
)", EXPERIMENTAL) \
|
||||
@ -5843,7 +5843,7 @@ If it is set to true, allow to use experimental full-text index.
|
||||
\
|
||||
DECLARE(Bool, allow_experimental_join_condition, false, R"(
|
||||
Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.
|
||||
)", 0) \
|
||||
)", EXPERIMENTAL) \
|
||||
\
|
||||
DECLARE(Bool, allow_experimental_live_view, false, R"(
|
||||
Allows creation of a deprecated LIVE VIEW.
|
||||
@ -5852,7 +5852,7 @@ Possible values:
|
||||
|
||||
- 0 — Working with live views is disabled.
|
||||
- 1 — Working with live views is enabled.
|
||||
)", 0) \
|
||||
)", EXPERIMENTAL) \
|
||||
DECLARE(Seconds, live_view_heartbeat_interval, 15, R"(
|
||||
The heartbeat interval in seconds to indicate live query is alive.
|
||||
)", EXPERIMENTAL) \
|
||||
@ -6206,6 +6206,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);
|
||||
|
@ -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;
|
||||
|
@ -4991,7 +4991,12 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi
|
||||
getAccessControl().setDefaultProfileName(shared->default_profile_name);
|
||||
|
||||
shared->system_profile_name = config.getString("system_profile", shared->default_profile_name);
|
||||
setCurrentProfile(shared->system_profile_name);
|
||||
|
||||
/// Don't check for constraints on first load. This makes the default profile consistent with other users, where
|
||||
/// the default value set in the config might be outside of the constraints range
|
||||
/// It makes it possible to change the value of experimental settings with `allowed_feature_tier` != 2
|
||||
bool check_constraints = false;
|
||||
setCurrentProfile(shared->system_profile_name, check_constraints);
|
||||
|
||||
applySettingsQuirks(*settings, getLogger("SettingsQuirks"));
|
||||
doSettingsSanityCheckClamp(*settings, getLogger("SettingsSanity"));
|
||||
|
@ -289,7 +289,10 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
|
||||
/// Use global context with fresh system profile settings
|
||||
auto system_context = Context::createCopy(getContext()->getGlobalContext());
|
||||
system_context->setSetting("profile", getContext()->getSystemProfileName());
|
||||
/// Don't check for constraints when changing profile. It was accepted before (for example it might include
|
||||
/// some experimental settings)
|
||||
bool check_constraints = false;
|
||||
system_context->setCurrentProfile(getContext()->getSystemProfileName(), check_constraints);
|
||||
|
||||
/// Make canonical query for simpler processing
|
||||
if (query.type == Type::RELOAD_DICTIONARY)
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
#include <Access/AccessControl.h>
|
||||
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||
#include <Analyzer/QueryTreeBuilder.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
@ -462,7 +463,12 @@ MergeTreeData::MergeTreeData(
|
||||
|
||||
/// Check sanity of MergeTreeSettings. Only when table is created.
|
||||
if (sanity_checks)
|
||||
settings->sanityCheck(getContext()->getMergeMutateExecutor()->getMaxTasksCount());
|
||||
{
|
||||
const auto & ac = getContext()->getAccessControl();
|
||||
bool allow_experimental = ac.getAllowExperimentalTierSettings();
|
||||
bool allow_beta = ac.getAllowBetaTierSettings();
|
||||
settings->sanityCheck(getContext()->getMergeMutateExecutor()->getMaxTasksCount(), allow_experimental, allow_beta);
|
||||
}
|
||||
|
||||
if (!date_column_name.empty())
|
||||
{
|
||||
@ -3896,7 +3902,10 @@ void MergeTreeData::changeSettings(
|
||||
/// Reset to default settings before applying existing.
|
||||
auto copy = getDefaultSettings();
|
||||
copy->applyChanges(new_changes);
|
||||
copy->sanityCheck(getContext()->getMergeMutateExecutor()->getMaxTasksCount());
|
||||
const auto & ac = getContext()->getAccessControl();
|
||||
bool allow_experimental = ac.getAllowExperimentalTierSettings();
|
||||
bool allow_beta = ac.getAllowBetaTierSettings();
|
||||
copy->sanityCheck(getContext()->getMergeMutateExecutor()->getMaxTasksCount(), allow_experimental, allow_beta);
|
||||
|
||||
storage_settings.set(std::move(copy));
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
|
@ -28,6 +28,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_SETTING;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int READONLY;
|
||||
}
|
||||
|
||||
// clang-format off
|
||||
@ -297,7 +298,7 @@ struct MergeTreeSettingsImpl : public BaseSettings<MergeTreeSettingsTraits>
|
||||
void loadFromQuery(ASTStorage & storage_def, ContextPtr context, bool is_attach);
|
||||
|
||||
/// Check that the values are sane taking also query-level settings into account.
|
||||
void sanityCheck(size_t background_pool_tasks) const;
|
||||
void sanityCheck(size_t background_pool_tasks, bool allow_experimental, bool allow_beta) const;
|
||||
};
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS)
|
||||
@ -377,8 +378,35 @@ void MergeTreeSettingsImpl::loadFromQuery(ASTStorage & storage_def, ContextPtr c
|
||||
#undef ADD_IF_ABSENT
|
||||
}
|
||||
|
||||
void MergeTreeSettingsImpl::sanityCheck(size_t background_pool_tasks) const
|
||||
void MergeTreeSettingsImpl::sanityCheck(size_t background_pool_tasks, bool allow_experimental, bool allow_beta) const
|
||||
{
|
||||
if (!allow_experimental || !allow_beta)
|
||||
{
|
||||
for (const auto & setting : all())
|
||||
{
|
||||
if (!setting.isValueChanged())
|
||||
continue;
|
||||
|
||||
auto tier = setting.getTier();
|
||||
if (!allow_experimental && tier == EXPERIMENTAL)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::READONLY,
|
||||
"Cannot modify setting '{}'. Changes to EXPERIMENTAL settings are disabled in the server config "
|
||||
"('allowed_feature_tier')",
|
||||
setting.getName());
|
||||
}
|
||||
if (!allow_beta && tier == BETA)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::READONLY,
|
||||
"Cannot modify setting '{}'. Changes to BETA settings are disabled in the server config ('allowed_feature_tier')",
|
||||
setting.getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (number_of_free_entries_in_pool_to_execute_mutation > background_pool_tasks)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_mutation' setting"
|
||||
@ -625,9 +653,9 @@ bool MergeTreeSettings::needSyncPart(size_t input_rows, size_t input_bytes) cons
|
||||
|| (impl->min_compressed_bytes_to_fsync_after_merge && input_bytes >= impl->min_compressed_bytes_to_fsync_after_merge));
|
||||
}
|
||||
|
||||
void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const
|
||||
void MergeTreeSettings::sanityCheck(size_t background_pool_tasks, bool allow_experimental, bool allow_beta) const
|
||||
{
|
||||
impl->sanityCheck(background_pool_tasks);
|
||||
impl->sanityCheck(background_pool_tasks, allow_experimental, allow_beta);
|
||||
}
|
||||
|
||||
void MergeTreeSettings::dumpToSystemMergeTreeSettingsColumns(MutableColumnsAndConstraints & params) const
|
||||
|
@ -77,7 +77,7 @@ struct MergeTreeSettings
|
||||
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
bool needSyncPart(size_t input_rows, size_t input_bytes) const;
|
||||
void sanityCheck(size_t background_pool_tasks) const;
|
||||
void sanityCheck(size_t background_pool_tasks, bool allow_experimental, bool allow_beta) const;
|
||||
|
||||
void dumpToSystemMergeTreeSettingsColumns(MutableColumnsAndConstraints & params) const;
|
||||
|
||||
|
@ -0,0 +1,3 @@
|
||||
<clickhouse>
|
||||
<allowed_feature_tier>0</allowed_feature_tier>
|
||||
</clickhouse>
|
@ -0,0 +1,17 @@
|
||||
<clickhouse>
|
||||
<profiles>
|
||||
<default>
|
||||
<allow_experimental_time_series_table>0</allow_experimental_time_series_table>
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
<default>
|
||||
<password></password>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
<named_collection_control>1</named_collection_control>
|
||||
<show_named_collections>1</show_named_collections>
|
||||
<show_named_collections_secrets>1</show_named_collections_secrets>
|
||||
</default>
|
||||
</users>
|
||||
</clickhouse>
|
295
tests/integration/test_allowed_feature_tier/test.py
Normal file
295
tests/integration/test_allowed_feature_tier/test.py
Normal file
@ -0,0 +1,295 @@
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance(
|
||||
"instance",
|
||||
main_configs=["configs/allowed_feature_tier.xml"],
|
||||
user_configs=[
|
||||
"configs/users.d/users.xml",
|
||||
],
|
||||
stay_alive=True,
|
||||
)
|
||||
|
||||
feature_tier_path = "/etc/clickhouse-server/config.d/allowed_feature_tier.xml"
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def get_current_tier_value(instance):
|
||||
query_with_current_tier_value = (
|
||||
"SELECT value FROM system.server_settings where name = 'allowed_feature_tier'"
|
||||
)
|
||||
return instance.query(query_with_current_tier_value).strip()
|
||||
|
||||
|
||||
def test_allowed_feature_tier_in_general_settings(start_cluster):
|
||||
# We use these settings as an example. If it fails in the future because you've changed the tier of the setting
|
||||
# please change it to another setting in the same tier. If there is none, feel free to comment out the test for that tier
|
||||
query_with_experimental_setting = (
|
||||
"SELECT 1 SETTINGS allow_experimental_time_series_table=1"
|
||||
)
|
||||
query_with_beta_setting = "SELECT 1 SETTINGS enable_parallel_replicas=1"
|
||||
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
query_with_experimental_setting
|
||||
)
|
||||
assert error == ""
|
||||
assert "1" == output.strip()
|
||||
|
||||
# Disable experimental settings
|
||||
instance.replace_in_config(feature_tier_path, "0", "1")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "1" == get_current_tier_value(instance)
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
query_with_experimental_setting
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(query_with_beta_setting)
|
||||
assert error == ""
|
||||
assert "1" == output.strip()
|
||||
|
||||
# Disable experimental and beta settings
|
||||
instance.replace_in_config(feature_tier_path, "1", "2")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "2" == get_current_tier_value(instance)
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
query_with_experimental_setting
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(query_with_beta_setting)
|
||||
assert output == ""
|
||||
assert "Changes to BETA settings are disabled" in error
|
||||
|
||||
# Leave the server as it was
|
||||
instance.replace_in_config(feature_tier_path, "2", "0")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
|
||||
|
||||
def test_allowed_feature_tier_in_mergetree_settings(start_cluster):
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
instance.query("DROP TABLE IF EXISTS test_experimental")
|
||||
|
||||
# Disable experimental settings
|
||||
instance.replace_in_config(feature_tier_path, "0", "1")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "1" == get_current_tier_value(instance)
|
||||
|
||||
query_with_experimental_mergetree_setting = """
|
||||
CREATE TABLE test_experimental (uid String, version UInt32, is_deleted UInt8)
|
||||
ENGINE = ReplacingMergeTree(version, is_deleted)
|
||||
ORDER by (uid)
|
||||
SETTINGS allow_experimental_replacing_merge_with_cleanup=1;
|
||||
"""
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
query_with_experimental_mergetree_setting
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
|
||||
# Go back
|
||||
instance.replace_in_config(feature_tier_path, "1", "0")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
query_with_experimental_mergetree_setting
|
||||
)
|
||||
assert output == ""
|
||||
assert error == ""
|
||||
|
||||
output = instance.query(
|
||||
"SELECT engine_full FROM system.tables WHERE name = 'test_experimental'"
|
||||
)
|
||||
assert "allow_experimental_replacing_merge_with_cleanup" in output
|
||||
|
||||
# We now disable experimental settings and restart the server to confirm it boots correctly
|
||||
instance.replace_in_config(feature_tier_path, "0", "1")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "1" == get_current_tier_value(instance)
|
||||
|
||||
instance.restart_clickhouse()
|
||||
|
||||
# After the reboot the table will be there
|
||||
output = instance.query(
|
||||
"SELECT engine_full FROM system.tables WHERE name = 'test_experimental'"
|
||||
)
|
||||
assert "allow_experimental_replacing_merge_with_cleanup" in output
|
||||
|
||||
# Creating a different table should not be possible
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"""
|
||||
CREATE TABLE test_experimental_new (uid String, version UInt32, is_deleted UInt8)
|
||||
ENGINE = ReplacingMergeTree(version, is_deleted)
|
||||
ORDER by (uid)
|
||||
SETTINGS allow_experimental_replacing_merge_with_cleanup=1;
|
||||
"""
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
|
||||
# Creating a different table and altering its settings to enable experimental should not be possible either
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"""
|
||||
CREATE TABLE test_experimental_new (uid String, version UInt32, is_deleted UInt8)
|
||||
ENGINE = ReplacingMergeTree(version, is_deleted)
|
||||
ORDER by (uid);
|
||||
"""
|
||||
)
|
||||
assert output == ""
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"""
|
||||
ALTER TABLE test_experimental_new MODIFY setting allow_experimental_replacing_merge_with_cleanup=1
|
||||
"""
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
instance.query("DROP TABLE IF EXISTS test_experimental_new")
|
||||
|
||||
instance.replace_in_config(feature_tier_path, "1", "0")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
instance.query("DROP TABLE IF EXISTS test_experimental")
|
||||
|
||||
|
||||
def test_allowed_feature_tier_in_user(start_cluster):
|
||||
instance.query("DROP USER IF EXISTS user_experimental")
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
|
||||
# Disable experimental settings
|
||||
instance.replace_in_config(feature_tier_path, "0", "1")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "1" == get_current_tier_value(instance)
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"CREATE USER user_experimental IDENTIFIED WITH no_password SETTINGS allow_experimental_time_series_table = 1"
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
|
||||
# Go back to normal and create the user to restart the server and verify it works
|
||||
instance.replace_in_config(feature_tier_path, "1", "0")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"CREATE USER user_experimental IDENTIFIED WITH no_password SETTINGS allow_experimental_time_series_table = 1"
|
||||
)
|
||||
assert output == ""
|
||||
assert error == ""
|
||||
|
||||
# Default user = 0
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT value FROM system.settings WHERE name = 'allow_experimental_time_series_table'"
|
||||
)
|
||||
assert output.strip() == "0"
|
||||
assert error == ""
|
||||
|
||||
# New user = 1
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT value FROM system.settings WHERE name = 'allow_experimental_time_series_table'",
|
||||
user="user_experimental",
|
||||
)
|
||||
assert output.strip() == "1"
|
||||
assert error == ""
|
||||
|
||||
# Change back to block experimental features and restart to confirm everything is working as expected (only new changes are blocked)
|
||||
instance.replace_in_config(feature_tier_path, "0", "1")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "1" == get_current_tier_value(instance)
|
||||
|
||||
instance.restart_clickhouse()
|
||||
|
||||
# Default user = 0
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT value FROM system.settings WHERE name = 'allow_experimental_time_series_table'"
|
||||
)
|
||||
assert output.strip() == "0"
|
||||
assert error == ""
|
||||
|
||||
# New user = 1
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT value FROM system.settings WHERE name = 'allow_experimental_time_series_table'",
|
||||
user="user_experimental",
|
||||
)
|
||||
assert output.strip() == "1"
|
||||
assert error == ""
|
||||
|
||||
# But note that they can't change the value either
|
||||
# 1 - 1 => OK
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT 1 SETTINGS allow_experimental_time_series_table=1",
|
||||
user="user_experimental",
|
||||
)
|
||||
assert output.strip() == "1"
|
||||
assert error == ""
|
||||
# 1 - 0 => KO
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT 1 SETTINGS allow_experimental_time_series_table=0",
|
||||
user="user_experimental",
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
|
||||
instance.replace_in_config(feature_tier_path, "1", "0")
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "0" == get_current_tier_value(instance)
|
||||
instance.query("DROP USER IF EXISTS user_experimental")
|
||||
|
||||
|
||||
def test_it_is_possible_to_enable_experimental_settings_in_default_profile(
|
||||
start_cluster,
|
||||
):
|
||||
# You can disable changing experimental settings but changing the default value via global config file is ok
|
||||
# It will just make the default value different and block changes
|
||||
instance.replace_in_config(feature_tier_path, "0", "2")
|
||||
|
||||
# Change default user config
|
||||
instance.replace_in_config(
|
||||
"/etc/clickhouse-server/users.d/users.xml",
|
||||
"allow_experimental_time_series_table>.",
|
||||
"allow_experimental_time_series_table>1",
|
||||
)
|
||||
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "2" == get_current_tier_value(instance)
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT value FROM system.settings WHERE name = 'allow_experimental_time_series_table'"
|
||||
)
|
||||
assert output.strip() == "1"
|
||||
assert error == ""
|
||||
|
||||
# But it won't be possible to change it
|
||||
output, error = instance.query_and_get_answer_with_error(
|
||||
"SELECT 1 SETTINGS allow_experimental_time_series_table=0"
|
||||
)
|
||||
assert output == ""
|
||||
assert "Changes to EXPERIMENTAL settings are disabled" in error
|
||||
|
||||
instance.replace_in_config(feature_tier_path, "2", "0")
|
||||
instance.replace_in_config(
|
||||
"/etc/clickhouse-server/users.d/users.xml",
|
||||
"allow_experimental_time_series_table>.",
|
||||
"allow_experimental_time_series_table>0",
|
||||
)
|
||||
|
||||
instance.query("SYSTEM RELOAD CONFIG")
|
||||
assert "0" == get_current_tier_value(instance)
|
Loading…
Reference in New Issue
Block a user