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:
Raúl Marín 2024-11-20 12:39:06 +00:00 committed by GitHub
commit 3ae055c74e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
20 changed files with 513 additions and 48 deletions

View File

@ -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).

View File

@ -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];

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]);
@ -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");

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

@ -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)
{

View File

@ -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.
*
*/

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) \
@ -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())
{

View File

@ -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);

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;

View File

@ -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"));

View File

@ -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)

View File

@ -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();

View File

@ -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

View File

@ -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;

View File

@ -0,0 +1,3 @@
<clickhouse>
<allowed_feature_tier>0</allowed_feature_tier>
</clickhouse>

View File

@ -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>

View 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)