mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #63185 from ClickHouse/backport/24.3/63119
Backport #63119 to 24.3: Make sanity check of settings worse
This commit is contained in:
commit
4897c7d7fd
@ -48,11 +48,6 @@ bool queryProfilerWorks() { return false; }
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_SETTING_VALUE;
|
||||
}
|
||||
|
||||
/// Update some settings defaults to avoid some known issues.
|
||||
void applySettingsQuirks(Settings & settings, LoggerPtr log)
|
||||
{
|
||||
@ -95,7 +90,7 @@ void applySettingsQuirks(Settings & settings, LoggerPtr log)
|
||||
}
|
||||
}
|
||||
|
||||
void doSettingsSanityCheck(const Settings & current_settings)
|
||||
void doSettingsSanityCheckClamp(Settings & current_settings, LoggerPtr log)
|
||||
{
|
||||
auto getCurrentValue = [¤t_settings](const std::string_view name) -> Field
|
||||
{
|
||||
@ -106,8 +101,13 @@ 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)
|
||||
{
|
||||
if (log)
|
||||
LOG_WARNING(log, "Sanity check: Too many threads requested ({}). Reduced to {}", 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 +122,11 @@ 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);
|
||||
{
|
||||
if (log)
|
||||
LOG_WARNING(log, "Sanity check: '{}' value is too high ({}). Reduced to {}", setting, block_size, max_sane_block_rows_size);
|
||||
current_settings.set(setting, max_sane_block_rows_size);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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, LoggerPtr log);
|
||||
}
|
||||
|
@ -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, getLogger("SettingsSanity"));
|
||||
}
|
||||
|
||||
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, getLogger("SettingsSanity"));
|
||||
}
|
||||
}
|
||||
|
||||
@ -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)
|
||||
@ -2074,7 +2074,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)
|
||||
@ -2087,7 +2087,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)
|
||||
@ -2107,7 +2107,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)
|
||||
{
|
||||
@ -2135,7 +2135,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)
|
||||
@ -2160,39 +2160,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, getLogger("SettingsSanity"));
|
||||
}
|
||||
|
||||
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, getLogger("SettingsSanity"));
|
||||
}
|
||||
|
||||
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, getLogger("SettingsSanity"));
|
||||
}
|
||||
|
||||
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, getLogger("SettingsSanity"));
|
||||
}
|
||||
|
||||
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, getLogger("SettingsSanity"));
|
||||
}
|
||||
|
||||
void Context::checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const
|
||||
@ -2200,32 +2200,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, getLogger("SettingsSanity"));
|
||||
}
|
||||
|
||||
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);
|
||||
@ -4457,7 +4457,7 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi
|
||||
setCurrentProfile(shared->system_profile_name);
|
||||
|
||||
applySettingsQuirks(settings, getLogger("SettingsQuirks"));
|
||||
doSettingsSanityCheck(settings);
|
||||
doSettingsSanityCheckClamp(settings, getLogger("SettingsSanity"));
|
||||
|
||||
shared->buffer_profile_name = config.getString("buffer_profile", shared->system_profile_name);
|
||||
buffer_context = Context::createCopy(shared_from_this());
|
||||
|
@ -781,11 +781,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
|
||||
@ -1291,15 +1291,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;
|
||||
|
||||
|
@ -92,7 +92,7 @@ public:
|
||||
|
||||
/// Just heuristic. We need one thread for collecting, one thread for receiving chunks
|
||||
/// and n threads for formatting.
|
||||
processing_units.resize(params.max_threads_for_parallel_formatting + 2);
|
||||
processing_units.resize(std::min(params.max_threads_for_parallel_formatting + 2, size_t{1024}));
|
||||
|
||||
/// Do not put any code that could throw an exception under this line.
|
||||
/// Because otherwise the destructor of this class won't be called and this thread won't be joined.
|
||||
|
@ -0,0 +1,19 @@
|
||||
0 0
|
||||
0 0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
Limit
|
||||
(ReadFromStorage)
|
||||
Zeros 0 → 1
|
||||
4294967296
|
@ -1,3 +1,4 @@
|
||||
SET send_logs_level = 'error';
|
||||
CREATE TABLE data_02052_1_wide0__fuzz_48
|
||||
(
|
||||
`key` Nullable(Int64),
|
||||
@ -13,12 +14,16 @@ 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';
|
||||
|
||||
-- Verify that we clamp odd values to something slightly saner
|
||||
SET max_block_size = 9223372036854775806;
|
||||
SELECT value FROM system.settings WHERE name = 'max_block_size';
|
||||
|
Loading…
Reference in New Issue
Block a user