Sanity check: Clamp values instead of throwing

This commit is contained in:
Raúl Marín 2024-04-29 14:34:47 +02:00
parent 379fea9a30
commit 97c781e365
6 changed files with 66 additions and 47 deletions

View File

@ -95,7 +95,7 @@ void applySettingsQuirks(Settings & settings, LoggerPtr log)
}
}
void doSettingsSanityCheck(const Settings & current_settings)
void doSettingsSanityCheckClamp(Settings & current_settings)
{
auto getCurrentValue = [&current_settings](const std::string_view name) -> Field
{
@ -106,8 +106,9 @@ void doSettingsSanityCheck(const Settings & current_settings)
};
UInt64 max_threads = getCurrentValue("max_threads").get<UInt64>();
if (max_threads > getNumberOfPhysicalCPUCores() * 65536)
throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Sanity check: Too many threads requested ({})", max_threads);
UInt64 max_threads_max_value = 256 * getNumberOfPhysicalCPUCores();
if (max_threads > max_threads_max_value)
current_settings.set("max_threads", max_threads_max_value);
constexpr UInt64 max_sane_block_rows_size = 4294967296; // 2^32
std::unordered_set<String> block_rows_settings{
@ -122,7 +123,7 @@ void doSettingsSanityCheck(const Settings & current_settings)
{
auto block_size = getCurrentValue(setting).get<UInt64>();
if (block_size > max_sane_block_rows_size)
throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Sanity check: '{}' value is too high ({})", setting, block_size);
current_settings.set(setting, max_sane_block_rows_size);
}
}
}

View File

@ -10,6 +10,6 @@ struct Settings;
/// Update some settings defaults to avoid some known issues.
void applySettingsQuirks(Settings & settings, LoggerPtr log = nullptr);
/// Verify that some settings have sane values. Throws if not
void doSettingsSanityCheck(const Settings & settings);
/// Verify that some settings have sane values. Alters the value to a reasonable one if not
void doSettingsSanityCheckClamp(Settings & settings);
}

View File

@ -1374,18 +1374,18 @@ std::shared_ptr<const EnabledRolesInfo> Context::getRolesInfo() const
namespace
{
ALWAYS_INLINE inline void
contextSanityCheckWithLock(const Context & context, const Settings & settings, const std::lock_guard<ContextSharedMutex> &)
contextSanityClampSettingsWithLock(const Context & context, Settings & settings, const std::lock_guard<ContextSharedMutex> &)
{
const auto type = context.getApplicationType();
if (type == Context::ApplicationType::LOCAL || type == Context::ApplicationType::SERVER)
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
ALWAYS_INLINE inline void contextSanityCheck(const Context & context, const Settings & settings)
ALWAYS_INLINE inline void contextSanityClampSettings(const Context & context, Settings & settings)
{
const auto type = context.getApplicationType();
if (type == Context::ApplicationType::LOCAL || type == Context::ApplicationType::SERVER)
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
}
@ -1498,7 +1498,7 @@ void Context::setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_i
checkSettingsConstraintsWithLock(profiles_info.settings, SettingSource::PROFILE);
applySettingsChangesWithLock(profiles_info.settings, lock);
settings_constraints_and_current_profiles = profiles_info.getConstraintsAndProfileIDs(settings_constraints_and_current_profiles);
contextSanityCheckWithLock(*this, settings, lock);
contextSanityClampSettingsWithLock(*this, settings, lock);
}
void Context::setCurrentProfile(const String & profile_name, bool check_constraints)
@ -2101,7 +2101,7 @@ void Context::setSettings(const Settings & settings_)
std::lock_guard lock(mutex);
settings = settings_;
need_recalculate_access = true;
contextSanityCheck(*this, settings);
contextSanityClampSettings(*this, settings);
}
void Context::setSettingWithLock(std::string_view name, const String & value, const std::lock_guard<ContextSharedMutex> & lock)
@ -2114,7 +2114,7 @@ void Context::setSettingWithLock(std::string_view name, const String & value, co
settings.set(name, value);
if (ContextAccessParams::dependsOnSettingName(name))
need_recalculate_access = true;
contextSanityCheckWithLock(*this, settings, lock);
contextSanityClampSettingsWithLock(*this, settings, lock);
}
void Context::setSettingWithLock(std::string_view name, const Field & value, const std::lock_guard<ContextSharedMutex> & lock)
@ -2134,7 +2134,7 @@ void Context::applySettingChangeWithLock(const SettingChange & change, const std
try
{
setSettingWithLock(change.name, change.value, lock);
contextSanityCheckWithLock(*this, settings, lock);
contextSanityClampSettingsWithLock(*this, settings, lock);
}
catch (Exception & e)
{
@ -2162,7 +2162,7 @@ void Context::setSetting(std::string_view name, const Field & value)
{
std::lock_guard lock(mutex);
setSettingWithLock(name, value, lock);
contextSanityCheckWithLock(*this, settings, lock);
contextSanityClampSettingsWithLock(*this, settings, lock);
}
void Context::applySettingChange(const SettingChange & change)
@ -2187,39 +2187,39 @@ void Context::applySettingsChanges(const SettingsChanges & changes)
applySettingsChangesWithLock(changes, lock);
}
void Context::checkSettingsConstraintsWithLock(const SettingsProfileElements & profile_elements, SettingSource source) const
void Context::checkSettingsConstraintsWithLock(const SettingsProfileElements & profile_elements, SettingSource source)
{
getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(settings, profile_elements, source);
if (getApplicationType() == ApplicationType::LOCAL || getApplicationType() == ApplicationType::SERVER)
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
void Context::checkSettingsConstraintsWithLock(const SettingChange & change, SettingSource source) const
void Context::checkSettingsConstraintsWithLock(const SettingChange & change, SettingSource source)
{
getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(settings, change, source);
if (getApplicationType() == ApplicationType::LOCAL || getApplicationType() == ApplicationType::SERVER)
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
void Context::checkSettingsConstraintsWithLock(const SettingsChanges & changes, SettingSource source) const
void Context::checkSettingsConstraintsWithLock(const SettingsChanges & changes, SettingSource source)
{
getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(settings, changes, source);
if (getApplicationType() == ApplicationType::LOCAL || getApplicationType() == ApplicationType::SERVER)
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
void Context::checkSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source) const
void Context::checkSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source)
{
getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(settings, changes, source);
if (getApplicationType() == ApplicationType::LOCAL || getApplicationType() == ApplicationType::SERVER)
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
void Context::clampToSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source) const
void Context::clampToSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source)
{
getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.clamp(settings, changes, source);
if (getApplicationType() == ApplicationType::LOCAL || getApplicationType() == ApplicationType::SERVER)
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
void Context::checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const
@ -2227,32 +2227,32 @@ void Context::checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings
getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(merge_tree_settings, changes);
}
void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const
void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source)
{
SharedLockGuard lock(mutex);
checkSettingsConstraintsWithLock(profile_elements, source);
}
void Context::checkSettingsConstraints(const SettingChange & change, SettingSource source) const
void Context::checkSettingsConstraints(const SettingChange & change, SettingSource source)
{
SharedLockGuard lock(mutex);
checkSettingsConstraintsWithLock(change, source);
}
void Context::checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const
void Context::checkSettingsConstraints(const SettingsChanges & changes, SettingSource source)
{
SharedLockGuard lock(mutex);
getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(settings, changes, source);
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
}
void Context::checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const
void Context::checkSettingsConstraints(SettingsChanges & changes, SettingSource source)
{
SharedLockGuard lock(mutex);
checkSettingsConstraintsWithLock(changes, source);
}
void Context::clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const
void Context::clampToSettingsConstraints(SettingsChanges & changes, SettingSource source)
{
SharedLockGuard lock(mutex);
clampToSettingsConstraintsWithLock(changes, source);
@ -4484,7 +4484,7 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi
setCurrentProfile(shared->system_profile_name);
applySettingsQuirks(settings, getLogger("SettingsQuirks"));
doSettingsSanityCheck(settings);
doSettingsSanityCheckClamp(settings);
shared->buffer_profile_name = config.getString("buffer_profile", shared->system_profile_name);
buffer_context = Context::createCopy(shared_from_this());

View File

@ -783,11 +783,11 @@ public:
void applySettingsChanges(const SettingsChanges & changes);
/// Checks the constraints.
void checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const;
void checkSettingsConstraints(const SettingChange & change, SettingSource source) const;
void checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const;
void checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const;
void clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const;
void checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source);
void checkSettingsConstraints(const SettingChange & change, SettingSource source);
void checkSettingsConstraints(const SettingsChanges & changes, SettingSource source);
void checkSettingsConstraints(SettingsChanges & changes, SettingSource source);
void clampToSettingsConstraints(SettingsChanges & changes, SettingSource source);
void checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const;
/// Reset settings to default value
@ -1293,15 +1293,15 @@ private:
void setCurrentDatabaseWithLock(const String & name, const std::lock_guard<ContextSharedMutex> & lock);
void checkSettingsConstraintsWithLock(const SettingsProfileElements & profile_elements, SettingSource source) const;
void checkSettingsConstraintsWithLock(const SettingsProfileElements & profile_elements, SettingSource source);
void checkSettingsConstraintsWithLock(const SettingChange & change, SettingSource source) const;
void checkSettingsConstraintsWithLock(const SettingChange & change, SettingSource source);
void checkSettingsConstraintsWithLock(const SettingsChanges & changes, SettingSource source) const;
void checkSettingsConstraintsWithLock(const SettingsChanges & changes, SettingSource source);
void checkSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source) const;
void checkSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source);
void clampToSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source) const;
void clampToSettingsConstraintsWithLock(SettingsChanges & changes, SettingSource source);
void checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const;

View File

@ -0,0 +1,18 @@
0 0
0 0
1
1
1
1
1
1
1
1
1
1
(Expression)
ExpressionTransform
(Limit)
Limit
(ReadFromStorage)
Zeros 0 → 1

View File

@ -13,12 +13,12 @@ FROM numbers(1);
SELECT * APPLY max
FROM data_02052_1_wide0__fuzz_48
GROUP BY toFixedString(toFixedString(toFixedString(toFixedString(toFixedString(toLowCardinality('UInt256'), toFixedString(toNullable(toNullable(2)), toFixedString(toFixedString(7), 7)), 7), 7), materialize(toNullable(7))), 7), materialize(7))
GROUP BY key
WITH CUBE
SETTINGS max_read_buffer_size = 7, max_threads = 9223372036854775807; -- { serverError INVALID_SETTING_VALUE }
SETTINGS max_read_buffer_size = 7, max_threads = 9223372036854775807;
SELECT zero + 1 AS x
FROM system.zeros
SETTINGS max_block_size = 9223372036854775806, max_rows_to_read = 20, read_overflow_mode = 'break'; -- { serverError INVALID_SETTING_VALUE }
FROM system.zeros LIMIT 10
SETTINGS max_block_size = 9223372036854775806, max_rows_to_read = 20, read_overflow_mode = 'break';
EXPLAIN PIPELINE SELECT zero + 1 AS x FROM system.zeros SETTINGS max_block_size = 9223372036854775806, max_rows_to_read = 20, read_overflow_mode = 'break'; -- { serverError INVALID_SETTING_VALUE }
EXPLAIN PIPELINE SELECT zero + 1 AS x FROM system.zeros LIMIT 10 SETTINGS max_block_size = 9223372036854775806, max_rows_to_read = 20, read_overflow_mode = 'break';