Merge branch 'master' into merge-tree-settings-sanity-check

This commit is contained in:
Alexey Milovidov 2020-08-02 17:13:17 +03:00
commit 778abb346f
88 changed files with 3616 additions and 2904 deletions

View File

@ -16,4 +16,4 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Events
* [ClickHouse at ByteDance (in Chinese)](https://mp.weixin.qq.com/s/Em-HjPylO8D7WPui4RREAQ) on July 31, 2020.
* [ClickHouse at ByteDance (in Chinese)](https://mp.weixin.qq.com/s/Em-HjPylO8D7WPui4RREAQ) on August 14, 2020.

View File

@ -54,7 +54,7 @@ endif ()
# Example: DwarfInstructions.hpp: register unsigned long long x16 __asm("x16") = cfa;
check_cxx_compiler_flag(-Wregister HAVE_WARNING_REGISTER)
if (HAVE_WARNING_REGISTER)
target_compile_options(unwind PRIVATE -Wno-register)
target_compile_options(unwind PRIVATE "$<$<STREQUAL:$<TARGET_PROPERTY:LANGUAGE>,CXX>:-Wno-register>")
endif ()
install(

View File

@ -67,6 +67,7 @@ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/conf
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/

View File

@ -17,6 +17,7 @@ ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/
ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/

View File

@ -232,10 +232,10 @@ private:
context.setQueryParameters(query_parameters);
/// settings and limits could be specified in config file, but passed settings has higher priority
for (const auto & setting : context.getSettingsRef())
for (auto setting : context.getSettingsRef().allUnchanged())
{
const String & name = setting.getName().toString();
if (config().has(name) && !setting.isChanged())
const auto & name = setting.getName();
if (config().has(name))
context.setSetting(name, config().getString(name));
}
@ -2252,9 +2252,9 @@ public:
/// Copy settings-related program options to config.
/// TODO: Is this code necessary?
for (const auto & setting : context.getSettingsRef())
for (auto setting : context.getSettingsRef().all())
{
const String name = setting.getName().toString();
const auto & name = setting.getName();
if (options.count(name))
config().setString(name, options[name].as<std::string>());
}

View File

@ -20,7 +20,6 @@
#include <Common/ThreadStatus.h>
#include <Common/config_version.h>
#include <Common/quoteString.h>
#include <Common/SettingsChanges.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/UseSSL.h>

View File

@ -223,9 +223,9 @@ void checkForUserSettingsAtTopLevel(const Poco::Util::AbstractConfiguration & co
return;
Settings settings;
for (const auto & setting : settings)
for (auto setting : settings.all())
{
std::string name = setting.getName().toString();
const auto & name = setting.getName();
if (config.has(name))
{
throw Exception(fmt::format("A setting '{}' appeared at top level in config {}."
@ -614,6 +614,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
global_context->setUncompressedCache(uncompressed_cache_size);
if (config().has("custom_settings_prefixes"))
global_context->getAccessControlManager().setCustomSettingsPrefixes(config().getString("custom_settings_prefixes"));
/// Load global settings from default_profile and system_profile.
global_context->setDefaultProfiles(config());
const Settings & settings = global_context->getSettingsRef();

View File

@ -262,6 +262,9 @@
<!-- Default profile of settings. -->
<default_profile>default</default_profile>
<!-- Comma-separated list of prefixes for user-defined settings. -->
<custom_settings_prefixes></custom_settings_prefixes>
<!-- System profile of settings. This settings are used by internal processes (Buffer storage, Distibuted DDL worker and so on). -->
<!-- <system_profile>default</system_profile> -->

View File

@ -11,12 +11,19 @@
#include <Access/SettingsProfilesCache.h>
#include <Access/ExternalAuthenticators.h>
#include <Core/Settings.h>
#include <common/find_symbols.h>
#include <Poco/ExpireCache.h>
#include <boost/algorithm/string/join.hpp>
#include <mutex>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
}
namespace
{
std::vector<std::unique_ptr<IAccessStorage>> createStorages()
@ -59,6 +66,53 @@ private:
};
class AccessControlManager::CustomSettingsPrefixes
{
public:
void registerPrefixes(const Strings & prefixes_)
{
std::lock_guard lock{mutex};
registered_prefixes = prefixes_;
}
bool isSettingNameAllowed(const std::string_view & setting_name) const
{
if (Settings::hasBuiltin(setting_name))
return true;
std::lock_guard lock{mutex};
for (const auto & prefix : registered_prefixes)
{
if (setting_name.starts_with(prefix))
return true;
}
return false;
}
void checkSettingNameIsAllowed(const std::string_view & setting_name) const
{
if (isSettingNameAllowed(setting_name))
return;
std::lock_guard lock{mutex};
if (!registered_prefixes.empty())
{
throw Exception(
"Setting " + String{setting_name} + " is neither a builtin setting nor started with the prefix '"
+ boost::algorithm::join(registered_prefixes, "' or '") + "' registered for user-defined settings",
ErrorCodes::UNKNOWN_SETTING);
}
else
BaseSettingsHelpers::throwSettingNotFound(setting_name);
}
private:
Strings registered_prefixes;
mutable std::mutex mutex;
};
AccessControlManager::AccessControlManager()
: MultipleAccessStorage(createStorages()),
context_access_cache(std::make_unique<ContextAccessCache>(*this)),
@ -66,7 +120,8 @@ AccessControlManager::AccessControlManager()
row_policy_cache(std::make_unique<RowPolicyCache>(*this)),
quota_cache(std::make_unique<QuotaCache>(*this)),
settings_profiles_cache(std::make_unique<SettingsProfilesCache>(*this)),
external_authenticators(std::make_unique<ExternalAuthenticators>())
external_authenticators(std::make_unique<ExternalAuthenticators>()),
custom_settings_prefixes(std::make_unique<CustomSettingsPrefixes>())
{
}
@ -100,6 +155,29 @@ void AccessControlManager::setDefaultProfileName(const String & default_profile_
}
void AccessControlManager::setCustomSettingsPrefixes(const Strings & prefixes)
{
custom_settings_prefixes->registerPrefixes(prefixes);
}
void AccessControlManager::setCustomSettingsPrefixes(const String & comma_separated_prefixes)
{
Strings prefixes;
splitInto<','>(prefixes, comma_separated_prefixes);
setCustomSettingsPrefixes(prefixes);
}
bool AccessControlManager::isSettingNameAllowed(const std::string_view & setting_name) const
{
return custom_settings_prefixes->isSettingNameAllowed(setting_name);
}
void AccessControlManager::checkSettingNameIsAllowed(const std::string_view & setting_name) const
{
custom_settings_prefixes->checkSettingNameIsAllowed(setting_name);
}
std::shared_ptr<const ContextAccess> AccessControlManager::getContextAccess(
const UUID & user_id,
const boost::container::flat_set<UUID> & current_roles,

View File

@ -1,7 +1,7 @@
#pragma once
#include <Access/MultipleAccessStorage.h>
#include <Poco/AutoPtr.h>
#include <Common/SettingsChanges.h>
#include <boost/container/flat_set.hpp>
#include <memory>
@ -53,6 +53,13 @@ public:
void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config);
void setDefaultProfileName(const String & default_profile_name);
/// Sets prefixes which should be used for custom settings.
/// This function also enables custom prefixes to be used.
void setCustomSettingsPrefixes(const Strings & prefixes);
void setCustomSettingsPrefixes(const String & comma_separated_prefixes);
bool isSettingNameAllowed(const std::string_view & name) const;
void checkSettingNameIsAllowed(const std::string_view & name) const;
std::shared_ptr<const ContextAccess> getContextAccess(
const UUID & user_id,
const boost::container::flat_set<UUID> & current_roles,
@ -89,14 +96,15 @@ public:
const ExternalAuthenticators & getExternalAuthenticators() const;
private:
class ContextAccessCache;
private: class ContextAccessCache;
class CustomSettingsPrefixes;
std::unique_ptr<ContextAccessCache> context_access_cache;
std::unique_ptr<RoleCache> role_cache;
std::unique_ptr<RowPolicyCache> row_policy_cache;
std::unique_ptr<QuotaCache> quota_cache;
std::unique_ptr<SettingsProfilesCache> settings_profiles_cache;
std::unique_ptr<ExternalAuthenticators> external_authenticators;
std::unique_ptr<CustomSettingsPrefixes> custom_settings_prefixes;
};
}

View File

@ -1,4 +1,5 @@
#include <Access/EnabledSettings.h>
#include <Common/SettingsChanges.h>
namespace DB

View File

@ -2,7 +2,6 @@
#include <Core/Types.h>
#include <Core/UUID.h>
#include <Common/SettingsChanges.h>
#include <Access/SettingsConstraints.h>
#include <Access/SettingsProfileElement.h>
#include <boost/container/flat_set.hpp>

View File

@ -1,22 +1,27 @@
#include <Access/SettingsConstraints.h>
#include <Access/AccessControlManager.h>
#include <Core/Settings.h>
#include <Common/FieldVisitors.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <IO/WriteHelpers.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <boost/range/algorithm_ext/erase.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int READONLY;
extern const int QUERY_IS_PROHIBITED;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int SETTING_CONSTRAINT_VIOLATION;
}
SettingsConstraints::SettingsConstraints() = default;
SettingsConstraints::SettingsConstraints(const AccessControlManager & manager_) : manager(&manager_)
{
}
SettingsConstraints::SettingsConstraints(const SettingsConstraints & src) = default;
SettingsConstraints & SettingsConstraints::operator=(const SettingsConstraints & src) = default;
SettingsConstraints::SettingsConstraints(SettingsConstraints && src) = default;
@ -26,28 +31,18 @@ SettingsConstraints::~SettingsConstraints() = default;
void SettingsConstraints::clear()
{
constraints_by_index.clear();
constraints.clear();
}
void SettingsConstraints::setMinValue(const StringRef & setting_name, const Field & min_value)
void SettingsConstraints::setMinValue(const std::string_view & setting_name, const Field & min_value)
{
setMinValue(Settings::findIndexStrict(setting_name), min_value);
getConstraintRef(setting_name).min_value = Settings::castValueUtil(setting_name, min_value);
}
void SettingsConstraints::setMinValue(size_t setting_index, const Field & min_value)
Field SettingsConstraints::getMinValue(const std::string_view & setting_name) const
{
getConstraintRef(setting_index).min_value = Settings::valueToCorrespondingType(setting_index, min_value);
}
Field SettingsConstraints::getMinValue(const StringRef & setting_name) const
{
return getMinValue(Settings::findIndexStrict(setting_name));
}
Field SettingsConstraints::getMinValue(size_t setting_index) const
{
const auto * ptr = tryGetConstraint(setting_index);
const auto * ptr = tryGetConstraint(setting_name);
if (ptr)
return ptr->min_value;
else
@ -55,24 +50,14 @@ Field SettingsConstraints::getMinValue(size_t setting_index) const
}
void SettingsConstraints::setMaxValue(const StringRef & name, const Field & max_value)
void SettingsConstraints::setMaxValue(const std::string_view & setting_name, const Field & max_value)
{
setMaxValue(Settings::findIndexStrict(name), max_value);
getConstraintRef(setting_name).max_value = Settings::castValueUtil(setting_name, max_value);
}
void SettingsConstraints::setMaxValue(size_t setting_index, const Field & max_value)
Field SettingsConstraints::getMaxValue(const std::string_view & setting_name) const
{
getConstraintRef(setting_index).max_value = Settings::valueToCorrespondingType(setting_index, max_value);
}
Field SettingsConstraints::getMaxValue(const StringRef & setting_name) const
{
return getMaxValue(Settings::findIndexStrict(setting_name));
}
Field SettingsConstraints::getMaxValue(size_t setting_index) const
{
const auto * ptr = tryGetConstraint(setting_index);
const auto * ptr = tryGetConstraint(setting_name);
if (ptr)
return ptr->max_value;
else
@ -80,51 +65,32 @@ Field SettingsConstraints::getMaxValue(size_t setting_index) const
}
void SettingsConstraints::setReadOnly(const StringRef & setting_name, bool read_only)
void SettingsConstraints::setReadOnly(const std::string_view & setting_name, bool read_only)
{
setReadOnly(Settings::findIndexStrict(setting_name), read_only);
getConstraintRef(setting_name).read_only = read_only;
}
void SettingsConstraints::setReadOnly(size_t setting_index, bool read_only)
bool SettingsConstraints::isReadOnly(const std::string_view & setting_name) const
{
getConstraintRef(setting_index).read_only = read_only;
}
bool SettingsConstraints::isReadOnly(const StringRef & setting_name) const
{
return isReadOnly(Settings::findIndexStrict(setting_name));
}
bool SettingsConstraints::isReadOnly(size_t setting_index) const
{
const auto * ptr = tryGetConstraint(setting_index);
const auto * ptr = tryGetConstraint(setting_name);
if (ptr)
return ptr->read_only;
else
return false;
}
void SettingsConstraints::set(const StringRef & setting_name, const Field & min_value, const Field & max_value, bool read_only)
{
set(Settings::findIndexStrict(setting_name), min_value, max_value, read_only);
}
void SettingsConstraints::set(size_t setting_index, const Field & min_value, const Field & max_value, bool read_only)
void SettingsConstraints::set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only)
{
auto & ref = getConstraintRef(setting_index);
ref.min_value = min_value;
ref.max_value = max_value;
auto & ref = getConstraintRef(setting_name);
ref.min_value = Settings::castValueUtil(setting_name, min_value);
ref.max_value = Settings::castValueUtil(setting_name, max_value);
ref.read_only = read_only;
}
void SettingsConstraints::get(const StringRef & setting_name, Field & min_value, Field & max_value, bool & read_only) const
void SettingsConstraints::get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const
{
get(Settings::findIndexStrict(setting_name), min_value, max_value, read_only);
}
void SettingsConstraints::get(size_t setting_index, Field & min_value, Field & max_value, bool & read_only) const
{
const auto * ptr = tryGetConstraint(setting_index);
const auto * ptr = tryGetConstraint(setting_name);
if (ptr)
{
min_value = ptr->min_value;
@ -141,9 +107,9 @@ void SettingsConstraints::get(size_t setting_index, Field & min_value, Field & m
void SettingsConstraints::merge(const SettingsConstraints & other)
{
for (const auto & [setting_index, other_constraint] : other.constraints_by_index)
for (const auto & [other_name, other_constraint] : other.constraints)
{
auto & constraint = constraints_by_index[setting_index];
auto & constraint = getConstraintRef(other_name);
if (!other_constraint.min_value.isNull())
constraint.min_value = other_constraint.min_value;
if (!other_constraint.max_value.isNull())
@ -154,95 +120,111 @@ void SettingsConstraints::merge(const SettingsConstraints & other)
}
SettingsConstraints::Infos SettingsConstraints::getInfo() const
{
Infos result;
result.reserve(constraints_by_index.size());
for (const auto & [setting_index, constraint] : constraints_by_index)
{
result.emplace_back();
Info & info = result.back();
info.name = Settings::getName(setting_index);
info.min = constraint.min_value;
info.max = constraint.max_value;
info.read_only = constraint.read_only;
}
return result;
}
void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change) const
{
const String & name = change.name;
size_t setting_index = Settings::findIndex(name);
if (setting_index == Settings::npos)
return;
Field new_value = Settings::valueToCorrespondingType(setting_index, change.value);
Field current_value = current_settings.get(setting_index);
/// Setting isn't checked if value wasn't changed.
if (current_value == new_value)
return;
if (!current_settings.allow_ddl && name == "allow_ddl")
throw Exception("Cannot modify 'allow_ddl' setting when DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
/** The `readonly` value is understood as follows:
* 0 - everything allowed.
* 1 - only read queries can be made; you can not change the settings.
* 2 - You can only do read queries and you can change the settings, except for the `readonly` setting.
*/
if (current_settings.readonly == 1)
throw Exception("Cannot modify '" + name + "' setting in readonly mode", ErrorCodes::READONLY);
if (current_settings.readonly > 1 && name == "readonly")
throw Exception("Cannot modify 'readonly' setting in readonly mode", ErrorCodes::READONLY);
const Constraint * constraint = tryGetConstraint(setting_index);
if (constraint)
{
if (constraint->read_only)
throw Exception("Setting " + name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
if (!constraint->min_value.isNull() && (new_value < constraint->min_value))
throw Exception(
"Setting " + name + " shouldn't be less than " + applyVisitor(FieldVisitorToString(), constraint->min_value),
ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
if (!constraint->max_value.isNull() && (new_value > constraint->max_value))
throw Exception(
"Setting " + name + " shouldn't be greater than " + applyVisitor(FieldVisitorToString(), constraint->max_value),
ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
}
checkImpl(current_settings, const_cast<SettingChange &>(change), THROW_ON_VIOLATION);
}
void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes) const
{
for (const auto & change : changes)
check(current_settings, change);
}
void SettingsConstraints::clamp(const Settings & current_settings, SettingChange & change) const
void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes) const
{
const String & name = change.name;
size_t setting_index = Settings::findIndex(name);
if (setting_index == Settings::npos)
return;
boost::range::remove_erase_if(
changes,
[&](SettingChange & change) -> bool
{
return !checkImpl(current_settings, const_cast<SettingChange &>(change), THROW_ON_VIOLATION);
});
}
Field new_value = Settings::valueToCorrespondingType(setting_index, change.value);
Field current_value = current_settings.get(setting_index);
void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const
{
boost::range::remove_erase_if(
changes,
[&](SettingChange & change) -> bool
{
return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION);
});
}
/// Setting isn't checked if value wasn't changed.
if (current_value == new_value)
return;
if (!current_settings.allow_ddl && name == "allow_ddl")
bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const
{
const String & setting_name = change.name;
bool cannot_cast;
auto cast_value = [&](const Field & x) -> Field
{
change.value = current_value;
return;
cannot_cast = false;
if (reaction == THROW_ON_VIOLATION)
return Settings::castValueUtil(setting_name, x);
else
{
try
{
return Settings::castValueUtil(setting_name, x);
}
catch (...)
{
cannot_cast = true;
return {};
}
}
};
bool cannot_compare = false;
auto less = [&](const Field & left, const Field & right)
{
cannot_compare = false;
if (reaction == THROW_ON_VIOLATION)
return applyVisitor(FieldVisitorAccurateLess{}, left, right);
else
{
try
{
return applyVisitor(FieldVisitorAccurateLess{}, left, right);
}
catch (...)
{
cannot_compare = true;
return false;
}
}
};
if (reaction == THROW_ON_VIOLATION)
manager->checkSettingNameIsAllowed(setting_name);
else if (!manager->isSettingNameAllowed(setting_name))
return false;
Field current_value, new_value;
if (current_settings.tryGet(setting_name, current_value))
{
/// Setting isn't checked if value has not changed.
if (change.value == current_value)
return false;
new_value = cast_value(change.value);
if ((new_value == current_value) || cannot_cast)
return false;
}
else
{
new_value = cast_value(change.value);
if (cannot_cast)
return false;
}
if (!current_settings.allow_ddl && setting_name == "allow_ddl")
{
if (reaction == THROW_ON_VIOLATION)
throw Exception("Cannot modify 'allow_ddl' setting when DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
else
return false;
}
/** The `readonly` value is understood as follows:
@ -252,125 +234,100 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingChange
*/
if (current_settings.readonly == 1)
{
change.value = current_value;
return;
if (reaction == THROW_ON_VIOLATION)
throw Exception("Cannot modify '" + setting_name + "' setting in readonly mode", ErrorCodes::READONLY);
else
return false;
}
if (current_settings.readonly > 1 && name == "readonly")
if (current_settings.readonly > 1 && setting_name == "readonly")
{
change.value = current_value;
return;
if (reaction == THROW_ON_VIOLATION)
throw Exception("Cannot modify 'readonly' setting in readonly mode", ErrorCodes::READONLY);
else
return false;
}
const Constraint * constraint = tryGetConstraint(setting_index);
const Constraint * constraint = tryGetConstraint(setting_name);
if (constraint)
{
if (constraint->read_only)
{
change.value = current_value;
return;
}
if (!constraint->min_value.isNull() && (new_value < constraint->min_value))
{
if (!constraint->max_value.isNull() && (constraint->min_value > constraint->max_value))
change.value = current_value;
if (reaction == THROW_ON_VIOLATION)
throw Exception("Setting " + setting_name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
else
change.value = constraint->min_value;
return;
return false;
}
if (!constraint->max_value.isNull() && (new_value > constraint->max_value))
const Field & min_value = constraint->min_value;
const Field & max_value = constraint->max_value;
if (!min_value.isNull() && !max_value.isNull() && (less(max_value, min_value) || cannot_compare))
{
change.value = constraint->max_value;
return;
if (reaction == THROW_ON_VIOLATION)
throw Exception("Setting " + setting_name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
else
return false;
}
if (!min_value.isNull() && (less(new_value, min_value) || cannot_compare))
{
if (reaction == THROW_ON_VIOLATION)
{
throw Exception(
"Setting " + setting_name + " shouldn't be less than " + applyVisitor(FieldVisitorToString(), constraint->min_value),
ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
}
else
change.value = min_value;
}
if (!max_value.isNull() && (less(max_value, new_value) || cannot_compare))
{
if (reaction == THROW_ON_VIOLATION)
{
throw Exception(
"Setting " + setting_name + " shouldn't be greater than " + applyVisitor(FieldVisitorToString(), constraint->max_value),
ErrorCodes::SETTING_CONSTRAINT_VIOLATION);
}
else
change.value = max_value;
}
}
return true;
}
void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const
SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(const std::string_view & setting_name)
{
for (auto & change : changes)
clamp(current_settings, change);
}
SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(size_t index)
{
auto it = constraints_by_index.find(index);
if (it == constraints_by_index.end())
it = constraints_by_index.emplace(index, Constraint{}).first;
auto it = constraints.find(setting_name);
if (it == constraints.end())
{
auto setting_name_ptr = std::make_shared<const String>(setting_name);
Constraint new_constraint;
new_constraint.setting_name = setting_name_ptr;
it = constraints.emplace(*setting_name_ptr, std::move(new_constraint)).first;
}
return it->second;
}
const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(size_t index) const
const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(const std::string_view & setting_name) const
{
auto it = constraints_by_index.find(index);
if (it == constraints_by_index.end())
auto it = constraints.find(setting_name);
if (it == constraints.end())
return nullptr;
return &it->second;
}
void SettingsConstraints::setProfile(const String & profile_name, const Poco::Util::AbstractConfiguration & config)
bool SettingsConstraints::Constraint::operator==(const Constraint & other) const
{
String elem = "profiles." + profile_name;
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(elem, config_keys);
for (const std::string & key : config_keys)
{
if (key == "profile" || key.starts_with("profile[")) /// Inheritance of profiles from the current one.
setProfile(config.getString(elem + "." + key), config);
else
continue;
}
String path_to_constraints = "profiles." + profile_name + ".constraints";
if (config.has(path_to_constraints))
loadFromConfig(path_to_constraints, config);
return (read_only == other.read_only) && (min_value == other.min_value) && (max_value == other.max_value)
&& (*setting_name == *other.setting_name);
}
void SettingsConstraints::loadFromConfig(const String & path_to_constraints, const Poco::Util::AbstractConfiguration & config)
bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right)
{
if (!config.has(path_to_constraints))
throw Exception("There is no path '" + path_to_constraints + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
Poco::Util::AbstractConfiguration::Keys names;
config.keys(path_to_constraints, names);
for (const String & name : names)
{
String path_to_name = path_to_constraints + "." + name;
Poco::Util::AbstractConfiguration::Keys constraint_types;
config.keys(path_to_name, constraint_types);
for (const String & constraint_type : constraint_types)
{
auto get_constraint_value = [&]{ return config.getString(path_to_name + "." + constraint_type); };
if (constraint_type == "min")
setMinValue(name, get_constraint_value());
else if (constraint_type == "max")
setMaxValue(name, get_constraint_value());
else if (constraint_type == "readonly")
setReadOnly(name, true);
else
throw Exception("Setting " + constraint_type + " value for " + name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED);
}
}
}
bool SettingsConstraints::Constraint::operator==(const Constraint & rhs) const
{
return (read_only == rhs.read_only) && (min_value == rhs.min_value) && (max_value == rhs.max_value);
}
bool operator ==(const SettingsConstraints & lhs, const SettingsConstraints & rhs)
{
return lhs.constraints_by_index == rhs.constraints_by_index;
return (left.constraints == right.constraints);
}
}

View File

@ -1,23 +1,21 @@
#pragma once
#include <Core/Field.h>
#include <Common/SettingsChanges.h>
#include <common/StringRef.h>
#include <unordered_map>
namespace Poco
{
namespace Util
namespace Poco::Util
{
class AbstractConfiguration;
}
}
namespace DB
{
struct Settings;
struct SettingChange;
class SettingsChanges;
class AccessControlManager;
/** Checks if specified changes of settings are allowed or not.
* If the changes are not allowed (i.e. violates some constraints) this class throws an exception.
@ -53,7 +51,7 @@ struct Settings;
class SettingsConstraints
{
public:
SettingsConstraints();
SettingsConstraints(const AccessControlManager & manager_);
SettingsConstraints(const SettingsConstraints & src);
SettingsConstraints & operator =(const SettingsConstraints & src);
SettingsConstraints(SettingsConstraints && src);
@ -61,75 +59,57 @@ public:
~SettingsConstraints();
void clear();
bool empty() const { return constraints_by_index.empty(); }
bool empty() const { return constraints.empty(); }
void setMinValue(const StringRef & setting_name, const Field & min_value);
void setMinValue(size_t setting_index, const Field & min_value);
Field getMinValue(const StringRef & setting_name) const;
Field getMinValue(size_t setting_index) const;
void setMinValue(const std::string_view & setting_name, const Field & min_value);
Field getMinValue(const std::string_view & setting_name) const;
void setMaxValue(const StringRef & setting_name, const Field & max_value);
void setMaxValue(size_t setting_index, const Field & max_value);
Field getMaxValue(const StringRef & setting_name) const;
Field getMaxValue(size_t setting_index) const;
void setMaxValue(const std::string_view & setting_name, const Field & max_value);
Field getMaxValue(const std::string_view & setting_name) const;
void setReadOnly(const StringRef & setting_name, bool read_only);
void setReadOnly(size_t setting_index, bool read_only);
bool isReadOnly(const StringRef & setting_name) const;
bool isReadOnly(size_t setting_index) const;
void setReadOnly(const std::string_view & setting_name, bool read_only);
bool isReadOnly(const std::string_view & setting_name) const;
void set(const StringRef & setting_name, const Field & min_value, const Field & max_value, bool read_only);
void set(size_t setting_index, const Field & min_value, const Field & max_value, bool read_only);
void get(const StringRef & setting_name, Field & min_value, Field & max_value, bool & read_only) const;
void get(size_t setting_index, Field & min_value, Field & max_value, bool & read_only) const;
void set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only);
void get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const;
void merge(const SettingsConstraints & other);
struct Info
{
StringRef name;
Field min;
Field max;
bool read_only = false;
};
using Infos = std::vector<Info>;
Infos getInfo() const;
/// Checks whether `change` violates these constraints and throws an exception if so.
void check(const Settings & current_settings, const SettingChange & change) const;
void check(const Settings & current_settings, const SettingsChanges & changes) const;
void check(const Settings & current_settings, SettingsChanges & changes) const;
/// Checks whether `change` violates these and clamps the `change` if so.
void clamp(const Settings & current_settings, SettingChange & change) const;
void clamp(const Settings & current_settings, SettingsChanges & changes) const;
/** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings).
* The profile can also be set using the `set` functions, like the profile setting.
*/
void setProfile(const String & profile_name, const Poco::Util::AbstractConfiguration & config);
/// Loads the constraints from configuration file, at "path" prefix in configuration.
void loadFromConfig(const String & path, const Poco::Util::AbstractConfiguration & config);
friend bool operator ==(const SettingsConstraints & lhs, const SettingsConstraints & rhs);
friend bool operator !=(const SettingsConstraints & lhs, const SettingsConstraints & rhs) { return !(lhs == rhs); }
friend bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right);
friend bool operator !=(const SettingsConstraints & left, const SettingsConstraints & right) { return !(left == right); }
private:
struct Constraint
{
std::shared_ptr<const String> setting_name;
bool read_only = false;
Field min_value;
Field max_value;
bool operator ==(const Constraint & rhs) const;
bool operator !=(const Constraint & rhs) const { return !(*this == rhs); }
bool operator ==(const Constraint & other) const;
bool operator !=(const Constraint & other) const { return !(*this == other); }
};
Constraint & getConstraintRef(size_t index);
const Constraint * tryGetConstraint(size_t) const;
enum ReactionOnViolation
{
THROW_ON_VIOLATION,
CLAMP_ON_VIOLATION,
};
bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const;
std::unordered_map<size_t, Constraint> constraints_by_index;
Constraint & getConstraintRef(const std::string_view & setting_name);
const Constraint * tryGetConstraint(const std::string_view & setting_name) const;
std::unordered_map<std::string_view, Constraint> constraints;
const AccessControlManager * manager = nullptr;
};
}

View File

@ -4,6 +4,7 @@
#include <Access/SettingsProfile.h>
#include <Parsers/ASTSettingsProfileElement.h>
#include <Core/Settings.h>
#include <Common/SettingsChanges.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
@ -35,18 +36,22 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A
if (!ast.setting_name.empty())
{
setting_index = Settings::findIndexStrict(ast.setting_name);
setting_name = ast.setting_name;
value = ast.value;
min_value = ast.min_value;
max_value = ast.max_value;
readonly = ast.readonly;
if (!value.isNull())
value = Settings::valueToCorrespondingType(setting_index, value);
value = Settings::castValueUtil(setting_name, value);
if (!min_value.isNull())
min_value = Settings::valueToCorrespondingType(setting_index, min_value);
min_value = Settings::castValueUtil(setting_name, min_value);
if (!max_value.isNull())
max_value = Settings::valueToCorrespondingType(setting_index, max_value);
max_value = Settings::castValueUtil(setting_name, max_value);
/// Optionally check if a setting with that name is allowed.
if (manager)
manager->checkSettingNameIsAllowed(setting_name);
}
}
@ -59,9 +64,7 @@ std::shared_ptr<ASTSettingsProfileElement> SettingsProfileElement::toAST() const
if (parent_profile)
ast->parent_profile = ::DB::toString(*parent_profile);
if (setting_index != static_cast<size_t>(-1))
ast->setting_name = Settings::getName(setting_index).toString();
ast->setting_name = setting_name;
ast->value = value;
ast->min_value = min_value;
ast->max_value = max_value;
@ -82,9 +85,7 @@ std::shared_ptr<ASTSettingsProfileElement> SettingsProfileElement::toASTWithName
ast->parent_profile = *parent_profile_name;
}
if (setting_index != static_cast<size_t>(-1))
ast->setting_name = Settings::getName(setting_index).toString();
ast->setting_name = setting_name;
ast->value = value;
ast->min_value = min_value;
ast->max_value = max_value;
@ -135,8 +136,8 @@ Settings SettingsProfileElements::toSettings() const
Settings res;
for (const auto & elem : *this)
{
if ((elem.setting_index != static_cast<size_t>(-1)) && !elem.value.isNull())
res.set(elem.setting_index, elem.value);
if (!elem.setting_name.empty() && !elem.value.isNull())
res.set(elem.setting_name, elem.value);
}
return res;
}
@ -146,25 +147,25 @@ SettingsChanges SettingsProfileElements::toSettingsChanges() const
SettingsChanges res;
for (const auto & elem : *this)
{
if ((elem.setting_index != static_cast<size_t>(-1)) && !elem.value.isNull())
res.push_back({Settings::getName(elem.setting_index).toString(), elem.value});
if (!elem.setting_name.empty() && !elem.value.isNull())
res.push_back({elem.setting_name, elem.value});
}
return res;
}
SettingsConstraints SettingsProfileElements::toSettingsConstraints() const
SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessControlManager & manager) const
{
SettingsConstraints res;
SettingsConstraints res{manager};
for (const auto & elem : *this)
{
if (elem.setting_index != static_cast<size_t>(-1))
if (!elem.setting_name.empty())
{
if (!elem.min_value.isNull())
res.setMinValue(elem.setting_index, elem.min_value);
res.setMinValue(elem.setting_name, elem.min_value);
if (!elem.max_value.isNull())
res.setMaxValue(elem.setting_index, elem.max_value);
res.setMaxValue(elem.setting_name, elem.max_value);
if (elem.readonly)
res.setReadOnly(elem.setting_index, *elem.readonly);
res.setReadOnly(elem.setting_name, *elem.readonly);
}
}
return res;

View File

@ -9,8 +9,7 @@
namespace DB
{
struct Settings;
struct SettingChange;
using SettingsChanges = std::vector<SettingChange>;
class SettingsChanges;
class SettingsConstraints;
class ASTSettingsProfileElement;
class ASTSettingsProfileElements;
@ -20,13 +19,13 @@ class AccessControlManager;
struct SettingsProfileElement
{
std::optional<UUID> parent_profile;
size_t setting_index = static_cast<size_t>(-1);
String setting_name;
Field value;
Field min_value;
Field max_value;
std::optional<bool> readonly;
auto toTuple() const { return std::tie(parent_profile, setting_index, value, min_value, max_value, readonly); }
auto toTuple() const { return std::tie(parent_profile, setting_name, value, min_value, max_value, readonly); }
friend bool operator==(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return lhs.toTuple() == rhs.toTuple(); }
friend bool operator!=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(lhs == rhs); }
friend bool operator <(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return lhs.toTuple() < rhs.toTuple(); }
@ -62,7 +61,7 @@ public:
Settings toSettings() const;
SettingsChanges toSettingsChanges() const;
SettingsConstraints toSettingsConstraints() const;
SettingsConstraints toSettingsConstraints(const AccessControlManager & manager) const;
};
}

View File

@ -2,6 +2,7 @@
#include <Access/AccessControlManager.h>
#include <Access/SettingsProfile.h>
#include <Core/Settings.h>
#include <Common/SettingsChanges.h>
#include <Common/quoteString.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
@ -102,6 +103,7 @@ void SettingsProfilesCache::setDefaultProfileName(const String & default_profile
default_profile_id = it->second;
}
void SettingsProfilesCache::mergeSettingsAndConstraints()
{
/// `mutex` is already locked.
@ -142,9 +144,10 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena
substituteProfiles(merged_settings);
auto settings = merged_settings.toSettings();
auto constraints = merged_settings.toSettingsConstraints(manager);
enabled.setSettingsAndConstraints(
std::make_shared<Settings>(merged_settings.toSettings()),
std::make_shared<SettingsConstraints>(merged_settings.toSettingsConstraints()));
std::make_shared<Settings>(std::move(settings)), std::make_shared<SettingsConstraints>(std::move(constraints)));
}

View File

@ -4,9 +4,9 @@
#include <Access/User.h>
#include <Access/SettingsProfile.h>
#include <Dictionaries/IDictionary.h>
#include <Core/Settings.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
#include <Core/Settings.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/MD5Engine.h>
#include <common/logger_useful.h>
@ -362,26 +362,25 @@ namespace
const String & path_to_constraints)
{
SettingsProfileElements profile_elements;
Poco::Util::AbstractConfiguration::Keys names;
config.keys(path_to_constraints, names);
for (const String & name : names)
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(path_to_constraints, keys);
for (const String & setting_name : keys)
{
SettingsProfileElement profile_element;
size_t setting_index = Settings::findIndexStrict(name);
profile_element.setting_index = setting_index;
profile_element.setting_name = setting_name;
Poco::Util::AbstractConfiguration::Keys constraint_types;
String path_to_name = path_to_constraints + "." + name;
String path_to_name = path_to_constraints + "." + setting_name;
config.keys(path_to_name, constraint_types);
for (const String & constraint_type : constraint_types)
{
if (constraint_type == "min")
profile_element.min_value = Settings::valueToCorrespondingType(setting_index, config.getString(path_to_name + "." + constraint_type));
profile_element.min_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type));
else if (constraint_type == "max")
profile_element.max_value = Settings::valueToCorrespondingType(setting_index, config.getString(path_to_name + "." + constraint_type));
profile_element.max_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type));
else if (constraint_type == "readonly")
profile_element.readonly = true;
else
throw Exception("Setting " + constraint_type + " value for " + name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Setting " + constraint_type + " value for " + setting_name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED);
}
profile_elements.push_back(std::move(profile_element));
}
@ -416,10 +415,10 @@ namespace
continue;
}
const auto & setting_name = key;
SettingsProfileElement profile_element;
size_t setting_index = Settings::findIndexStrict(key);
profile_element.setting_index = setting_index;
profile_element.value = Settings::valueToCorrespondingType(setting_index, config.getString(profile_config + "." + key));
profile_element.setting_name = setting_name;
profile_element.value = Settings::stringToValueUtil(setting_name, config.getString(profile_config + "." + key));
profile->elements.emplace_back(std::move(profile_element));
}

View File

@ -397,9 +397,9 @@ void Connection::sendQuery(
/// Per query settings.
if (settings)
{
auto settings_format = (server_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS
: SettingsBinaryFormat::OLD;
settings->serialize(*out, settings_format);
auto settings_format = (server_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS
: SettingsWriteFormat::BINARY;
settings->write(*out, settings_format);
}
else
writeStringBinary("" /* empty string is a marker of the end of settings */, *out);

View File

@ -69,12 +69,12 @@ public:
Field operator[](size_t index) const override
{
return String(reinterpret_cast<const char *>(&chars[n * index]), n);
return Field{&chars[n * index], n};
}
void get(size_t index, Field & res) const override
{
res.assignString(reinterpret_cast<const char *>(&chars[n * index]), n);
res = std::string_view{reinterpret_cast<const char *>(&chars[n * index]), n};
}
StringRef getDataAt(size_t index) const override

View File

@ -86,7 +86,7 @@ public:
void get(size_t n, Field & res) const override
{
assert(n < size());
res.assignString(&chars[offsetAt(n)], sizeAt(n) - 1);
res = std::string_view{reinterpret_cast<const char *>(&chars[offsetAt(n)]), sizeAt(n) - 1};
}
StringRef getDataAt(size_t n) const override

View File

@ -500,6 +500,7 @@ namespace ErrorCodes
extern const int INCONSISTENT_RESERVATIONS = 533;
extern const int NO_RESERVATIONS_PROVIDED = 534;
extern const int UNKNOWN_RAID_TYPE = 535;
extern const int CANNOT_RESTORE_FROM_FIELD_DUMP = 536;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -40,7 +40,6 @@ static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
writeChar('\'', buf);
}
String FieldVisitorDump::operator() (const Null &) const { return "NULL"; }
String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); }
String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); }
@ -93,8 +92,11 @@ String FieldVisitorDump::operator() (const Tuple & x) const
String FieldVisitorDump::operator() (const AggregateFunctionStateData & x) const
{
WriteBufferFromOwnString wb;
wb << "AggregateFunctionState_(";
writeQuoted(x.name, wb);
wb << ", ";
writeQuoted(x.data, wb);
wb << ')';
return wb.str();
}

View File

@ -0,0 +1,50 @@
#include <Common/SettingsChanges.h>
namespace DB
{
namespace
{
SettingChange * find(SettingsChanges & changes, const std::string_view & name)
{
auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; });
if (it == changes.end())
return nullptr;
return &*it;
}
const SettingChange * find(const SettingsChanges & changes, const std::string_view & name)
{
auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; });
if (it == changes.end())
return nullptr;
return &*it;
}
}
bool SettingsChanges::tryGet(const std::string_view & name, Field & out_value) const
{
const auto * change = find(*this, name);
if (!change)
return false;
out_value = change->value;
return true;
}
const Field * SettingsChanges::tryGet(const std::string_view & name) const
{
const auto * change = find(*this, name);
if (!change)
return nullptr;
return &change->value;
}
Field * SettingsChanges::tryGet(const std::string_view & name)
{
auto * change = find(*this, name);
if (!change)
return nullptr;
return &change->value;
}
}

View File

@ -5,21 +5,28 @@
namespace DB
{
struct SettingChange
{
String name;
Field value;
SettingChange() {}
SettingChange(const String & name_, const Field value_)
: name(name_)
, value(value_) {}
SettingChange() {}
SettingChange(const std::string_view & name_, const Field & value_) : name(name_), value(value_) {}
SettingChange(const std::string_view & name_, Field && value_) : name(name_), value(std::move(value_)) {}
friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); }
friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); }
};
using SettingsChanges = std::vector<SettingChange>;
class SettingsChanges : public std::vector<SettingChange>
{
public:
using std::vector<SettingChange>::vector;
bool tryGet(const std::string_view & name, Field & out_value) const;
const Field * tryGet(const std::string_view & name) const;
Field * tryGet(const std::string_view & name);
};
}

View File

@ -235,3 +235,21 @@ std::string trim(const std::string & str, F && predicate)
return str.substr(cut_front, size - cut_front - cut_back);
}
inline void trimLeft(std::string_view & str, char c = ' ')
{
while (str.starts_with(c))
str.remove_prefix(1);
}
inline void trimRight(std::string_view & str, char c = ' ')
{
while (str.ends_with(c))
str.remove_suffix(1);
}
inline void trim(std::string_view & str, char c = ' ')
{
trimLeft(str, c);
trimRight(str, c);
}

View File

@ -1,5 +1,6 @@
#include <Common/ThreadPool.h>
#include <Common/Exception.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <cassert>
#include <type_traits>
@ -24,6 +25,13 @@ namespace CurrentMetrics
}
template <typename Thread>
ThreadPoolImpl<Thread>::ThreadPoolImpl()
: ThreadPoolImpl(getNumberOfPhysicalCPUCores())
{
}
template <typename Thread>
ThreadPoolImpl<Thread>::ThreadPoolImpl(size_t max_threads_)
: ThreadPoolImpl(max_threads_, max_threads_, max_threads_)

View File

@ -29,6 +29,9 @@ class ThreadPoolImpl
public:
using Job = std::function<void()>;
/// Maximum number of threads is based on the number of physical cores.
ThreadPoolImpl();
/// Size is constant. Up to num_threads are created on demand and then run until shutdown.
explicit ThreadPoolImpl(size_t max_threads_);

View File

@ -4,7 +4,7 @@
#include <Common/ProfileEvents.h>
#include <Common/MemoryTracker.h>
#include <Core/SettingsCollection.h>
#include <Core/SettingsEnums.h>
#include <IO/Progress.h>

View File

@ -9,29 +9,33 @@
unsigned getNumberOfPhysicalCPUCores()
{
#if USE_CPUID
cpu_raw_data_t raw_data;
cpu_id_t data;
static const unsigned number = []
{
# if USE_CPUID
cpu_raw_data_t raw_data;
cpu_id_t data;
/// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method.
/// Also, libcpuid does not support some CPUs like AMD Hygon C86 7151.
if (0 != cpuid_get_raw_data(&raw_data) || 0 != cpu_identify(&raw_data, &data) || data.num_logical_cpus == 0)
/// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method.
/// Also, libcpuid does not support some CPUs like AMD Hygon C86 7151.
if (0 != cpuid_get_raw_data(&raw_data) || 0 != cpu_identify(&raw_data, &data) || data.num_logical_cpus == 0)
return std::thread::hardware_concurrency();
unsigned res = data.num_cores * data.total_logical_cpus / data.num_logical_cpus;
/// Also, libcpuid gives strange result on Google Compute Engine VMs.
/// Example:
/// num_cores = 12, /// number of physical cores on current CPU socket
/// total_logical_cpus = 1, /// total number of logical cores on all sockets
/// num_logical_cpus = 24. /// number of logical cores on current CPU socket
/// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1.
if (res != 0)
return res;
# endif
/// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system.
/// (Actually, only Aarch64 is supported).
return std::thread::hardware_concurrency();
unsigned res = data.num_cores * data.total_logical_cpus / data.num_logical_cpus;
/// Also, libcpuid gives strange result on Google Compute Engine VMs.
/// Example:
/// num_cores = 12, /// number of physical cores on current CPU socket
/// total_logical_cpus = 1, /// total number of logical cores on all sockets
/// num_logical_cpus = 24. /// number of logical cores on current CPU socket
/// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1.
if (res != 0)
return res;
#endif
/// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system.
/// (Actually, only Aarch64 is supported).
return std::thread::hardware_concurrency();
}();
return number;
}

View File

@ -75,6 +75,7 @@ SRCS(
RWLock.cpp
SensitiveDataMasker.cpp
setThreadName.cpp
SettingsChanges.cpp
SharedLibrary.cpp
ShellCommand.cpp
StackTrace.cpp

54
src/Core/BaseSettings.cpp Normal file
View File

@ -0,0 +1,54 @@
#include <Core/BaseSettings.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_SETTING;
}
void BaseSettingsHelpers::writeString(const std::string_view & str, WriteBuffer & out)
{
writeStringBinary(str, out);
}
String BaseSettingsHelpers::readString(ReadBuffer & in)
{
String str;
readStringBinary(str, in);
return str;
}
void BaseSettingsHelpers::writeFlags(Flags flags, WriteBuffer & out)
{
writeVarUInt(flags, out);
}
BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in)
{
UInt64 res;
readVarUInt(res, in);
return static_cast<Flags>(res);
}
void BaseSettingsHelpers::throwSettingNotFound(const std::string_view & name)
{
throw Exception("Unknown setting " + String{name}, ErrorCodes::UNKNOWN_SETTING);
}
void BaseSettingsHelpers::warningSettingNotFound(const std::string_view & name)
{
static auto * log = &Poco::Logger::get("Settings");
LOG_WARNING(log, "Unknown setting {}, skipping", name);
}
}

846
src/Core/BaseSettings.h Normal file
View File

@ -0,0 +1,846 @@
#pragma once
#include <Core/SettingsFields.h>
#include <Common/SettingsChanges.h>
#include <Common/FieldVisitors.h>
#include <ext/range.h>
#include <boost/blank.hpp>
#include <unordered_map>
namespace DB
{
class ReadBuffer;
class WriteBuffer;
enum class SettingsWriteFormat
{
BINARY, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour.
STRINGS_WITH_FLAGS, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized.
DEFAULT = STRINGS_WITH_FLAGS,
};
/** Template class to define collections of settings.
* Example of usage:
*
* mysettings.h:
* #define APPLY_FOR_MYSETTINGS(M) \
* M(UInt64, a, 100, "Description of a", 0) \
* M(Float, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \
* M(String, s, "default", "Description of s", 0)
*
* DECLARE_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
* struct MySettings : public BaseSettings<MySettingsTraits>
* {
* };
*
* mysettings.cpp:
* IMPLEMENT_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS)
*/
template <class Traits_>
class BaseSettings : public Traits_::Data
{
using CustomSettingMap = std::unordered_map<std::string_view, std::pair<std::shared_ptr<const String>, SettingFieldCustom>>;
public:
using Traits = Traits_;
void set(const std::string_view & name, const Field & value);
Field get(const std::string_view & name) const;
void setString(const std::string_view & name, const String & value);
String getString(const std::string_view & name) const;
bool tryGet(const std::string_view & name, Field & value) const;
bool tryGetString(const std::string_view & name, String & value) const;
bool isChanged(const std::string_view & name) const;
SettingsChanges changes() const;
void applyChange(const SettingChange & change);
void applyChanges(const SettingsChanges & changes);
void applyChanges(const BaseSettings & changes);
/// Resets all the settings to their default values.
void resetToDefault();
bool has(const std::string_view & name) const { return hasBuiltin(name) || hasCustom(name); }
static bool hasBuiltin(const std::string_view & name);
bool hasCustom(const std::string_view & name) const;
const char * getTypeName(const std::string_view & name) const;
const char * getDescription(const 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.
static void checkCanSet(const std::string_view & name, const Field & value);
static void checkCanSetString(const std::string_view & name, const String & str);
/// Conversions without changing the fields.
static Field castValueUtil(const std::string_view & name, const Field & value);
static String valueToStringUtil(const std::string_view & name, const Field & value);
static Field stringToValueUtil(const std::string_view & name, const String & str);
void write(WriteBuffer & out, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT) const;
void read(ReadBuffer & in, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT);
// A debugging aid.
std::string toString() const;
/// Represents a reference to a setting field.
class SettingFieldRef
{
public:
const String & getName() const;
Field getValue() const;
String getValueString() const;
bool isValueChanged() const;
const char * getTypeName() const;
const char * getDescription() const;
bool isCustom() const;
bool operator==(const SettingFieldRef & other) const { return (getName() == other.getName()) && (getValue() == other.getValue()); }
bool operator!=(const SettingFieldRef & other) const { return !(*this == other); }
private:
friend class BaseSettings;
SettingFieldRef(const typename Traits::Data & data_, const typename Traits::Accessor & accessor_, size_t index_) : data(&data_), accessor(&accessor_), index(index_) {}
SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_);
const typename Traits::Data * data = nullptr;
const typename Traits::Accessor * accessor = nullptr;
size_t index = 0;
std::conditional_t<Traits::allow_custom_settings, const CustomSettingMap::mapped_type*, boost::blank> custom_setting = {};
};
enum SkipFlags
{
SKIP_NONE = 0,
SKIP_CHANGED = 0x01,
SKIP_UNCHANGED = 0x02,
SKIP_BUILTIN = 0x04,
SKIP_CUSTOM = 0x08,
SKIP_ALL = SKIP_CHANGED | SKIP_UNCHANGED | SKIP_CUSTOM | SKIP_BUILTIN,
};
class Iterator
{
public:
Iterator & operator++();
Iterator operator++(int);
SettingFieldRef operator *() const;
bool operator ==(const Iterator & other) const;
bool operator !=(const Iterator & other) const { return !(*this == other); }
private:
friend class BaseSettings;
Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_);
void doSkip();
const BaseSettings * settings = nullptr;
const typename Traits::Accessor * accessor = nullptr;
size_t index;
std::conditional_t<Traits::allow_custom_settings, CustomSettingMap::const_iterator, boost::blank> custom_settings_iterator;
SkipFlags skip_flags;
};
class Range
{
public:
Range(const BaseSettings & settings_, SkipFlags skip_flags_) : settings(settings_), accessor(Traits::Accessor::instance()), skip_flags(skip_flags_) {}
Iterator begin() const { return Iterator(settings, accessor, skip_flags); }
Iterator end() const { return Iterator(settings, accessor, SKIP_ALL); }
private:
const BaseSettings & settings;
const typename Traits::Accessor & accessor;
SkipFlags skip_flags;
};
Range all(SkipFlags skip_flags = SKIP_NONE) const { return Range{*this, skip_flags}; }
Range allChanged() const { return all(SKIP_UNCHANGED); }
Range allUnchanged() const { return all(SKIP_CHANGED); }
Range allBuiltin() const { return all(SKIP_CUSTOM); }
Range allCustom() const { return all(SKIP_BUILTIN); }
Iterator begin() const { return allChanged().begin(); }
Iterator end() const { return allChanged().end(); }
private:
SettingFieldCustom & getCustomSetting(const std::string_view & name);
const SettingFieldCustom & getCustomSetting(const std::string_view & name) const;
const SettingFieldCustom * tryGetCustomSetting(const std::string_view & name) const;
std::conditional_t<Traits::allow_custom_settings, CustomSettingMap, boost::blank> custom_settings_map;
};
struct BaseSettingsHelpers
{
[[noreturn]] static void throwSettingNotFound(const std::string_view & name);
static void warningSettingNotFound(const std::string_view & name);
static void writeString(const std::string_view & str, WriteBuffer & out);
static String readString(ReadBuffer & in);
enum Flags : UInt64
{
IMPORTANT = 0x01,
CUSTOM = 0x02,
};
static void writeFlags(Flags flags, WriteBuffer & out);
static Flags readFlags(ReadBuffer & in);
};
template <typename Traits_>
void BaseSettings<Traits_>::set(const std::string_view & name, const Field & value)
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
accessor.setValue(*this, index, value);
else
getCustomSetting(name) = value;
}
template <typename Traits_>
Field BaseSettings<Traits_>::get(const std::string_view & name) const
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.getValue(*this, index);
else
return static_cast<Field>(getCustomSetting(name));
}
template <typename Traits_>
void BaseSettings<Traits_>::setString(const std::string_view & name, const String & value)
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
accessor.setValueString(*this, index, value);
else
getCustomSetting(name).parseFromString(value);
}
template <typename Traits_>
String BaseSettings<Traits_>::getString(const std::string_view & name) const
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.getValueString(*this, index);
else
return getCustomSetting(name).toString();
}
template <typename Traits_>
bool BaseSettings<Traits_>::tryGet(const std::string_view & name, Field & value) const
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
{
value = accessor.getValue(*this, index);
return true;
}
if (const auto * custom_setting = tryGetCustomSetting(name))
{
value = static_cast<Field>(*custom_setting);
return true;
}
return false;
}
template <typename Traits_>
bool BaseSettings<Traits_>::tryGetString(const std::string_view & name, String & value) const
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
{
value = accessor.getValueString(*this, index);
return true;
}
if (const auto * custom_setting = tryGetCustomSetting(name))
{
value = custom_setting->toString();
return true;
}
return false;
}
template <typename Traits_>
bool BaseSettings<Traits_>::isChanged(const std::string_view & name) const
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.isValueChanged(*this, index);
return tryGetCustomSetting(name) != nullptr;
}
template <typename Traits_>
SettingsChanges BaseSettings<Traits_>::changes() const
{
SettingsChanges res;
for (const auto & field : *this)
res.emplace_back(field.getName(), field.getValue());
return res;
}
template <typename Traits_>
void BaseSettings<Traits_>::applyChange(const SettingChange & change)
{
set(change.name, change.value);
}
template <typename Traits_>
void BaseSettings<Traits_>::applyChanges(const SettingsChanges & changes)
{
for (const auto & change : changes)
applyChange(change);
}
template <typename Traits_>
void BaseSettings<Traits_>::applyChanges(const BaseSettings & other_settings)
{
for (const auto & field : other_settings)
set(field.getName(), field.getValue());
}
template <typename Traits_>
void BaseSettings<Traits_>::resetToDefault()
{
const auto & accessor = Traits::Accessor::instance();
for (size_t i : ext::range(accessor.size()))
{
if (accessor.isValueChanged(*this, i))
accessor.resetValueToDefault(*this, i);
}
if constexpr (Traits::allow_custom_settings)
custom_settings_map.clear();
}
template <typename Traits_>
bool BaseSettings<Traits_>::hasBuiltin(const std::string_view & name)
{
const auto & accessor = Traits::Accessor::instance();
return (accessor.find(name) != static_cast<size_t>(-1));
}
template <typename Traits_>
bool BaseSettings<Traits_>::hasCustom(const std::string_view & name) const
{
return tryGetCustomSetting(name);
}
template <typename Traits_>
const char * BaseSettings<Traits_>::getTypeName(const std::string_view & name) const
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.getTypeName(index);
else if (tryGetCustomSetting(name))
return "Custom";
else
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename Traits_>
const char * BaseSettings<Traits_>::getDescription(const std::string_view & name) const
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.getDescription(index);
else if (tryGetCustomSetting(name))
return "Custom";
else
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename Traits_>
void BaseSettings<Traits_>::checkCanSet(const std::string_view & name, const Field & value)
{
castValueUtil(name, value);
}
template <typename Traits_>
void BaseSettings<Traits_>::checkCanSetString(const std::string_view & name, const String & str)
{
stringToValueUtil(name, str);
}
template <typename Traits_>
Field BaseSettings<Traits_>::castValueUtil(const std::string_view & name, const Field & value)
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.castValueUtil(index, value);
if constexpr (Traits::allow_custom_settings)
return value;
else
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename Traits_>
String BaseSettings<Traits_>::valueToStringUtil(const std::string_view & name, const Field & value)
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.valueToStringUtil(index, value);
if constexpr (Traits::allow_custom_settings)
return value.dump();
else
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename Traits_>
Field BaseSettings<Traits_>::stringToValueUtil(const std::string_view & name, const String & str)
{
const auto & accessor = Traits::Accessor::instance();
if (size_t index = accessor.find(name); index != static_cast<size_t>(-1))
return accessor.stringToValueUtil(index, str);
if constexpr (Traits::allow_custom_settings)
return Field::restoreFromDump(str);
else
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename Traits_>
void BaseSettings<Traits_>::write(WriteBuffer & out, SettingsWriteFormat format) const
{
const auto & accessor = Traits::Accessor::instance();
for (auto field : *this)
{
bool is_custom = field.isCustom();
bool is_important = !is_custom && accessor.isImportant(field.index);
BaseSettingsHelpers::writeString(field.getName(), out);
if ((format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) || is_custom)
{
using Flags = BaseSettingsHelpers::Flags;
Flags flags{0};
if (is_custom)
flags = static_cast<Flags>(flags | Flags::CUSTOM);
else if (is_important)
flags = static_cast<Flags>(flags | Flags::IMPORTANT);
BaseSettingsHelpers::writeFlags(flags, out);
BaseSettingsHelpers::writeString(field.getValueString(), out);
}
else
accessor.writeBinary(*this, field.index, out);
}
/// Empty string is a marker of the end of settings.
BaseSettingsHelpers::writeString(std::string_view{}, out);
}
template <typename Traits_>
void BaseSettings<Traits_>::read(ReadBuffer & in, SettingsWriteFormat format)
{
resetToDefault();
const auto & accessor = Traits::Accessor::instance();
while (true)
{
String name = BaseSettingsHelpers::readString(in);
if (name.empty() /* empty string is a marker of the end of settings */)
break;
size_t index = accessor.find(name);
using Flags = BaseSettingsHelpers::Flags;
Flags flags{0};
if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS)
flags = BaseSettingsHelpers::readFlags(in);
bool is_important = (flags & Flags::IMPORTANT);
bool is_custom = (flags & Flags::CUSTOM);
if (index != static_cast<size_t>(-1))
{
if (is_custom)
{
SettingFieldCustom temp;
temp.parseFromString(BaseSettingsHelpers::readString(in));
accessor.setValue(*this, index, static_cast<Field>(temp));
}
else if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS)
accessor.setValueString(*this, index, BaseSettingsHelpers::readString(in));
else
accessor.readBinary(*this, index, in);
}
else if (is_custom && Traits::allow_custom_settings)
{
getCustomSetting(name).parseFromString(BaseSettingsHelpers::readString(in));
}
else if (is_important)
{
BaseSettingsHelpers::throwSettingNotFound(name);
}
else
{
BaseSettingsHelpers::warningSettingNotFound(name);
BaseSettingsHelpers::readString(in);
}
}
}
template <typename Traits_>
String BaseSettings<Traits_>::toString() const
{
String res;
for (const auto & field : *this)
{
if (!res.empty())
res += ", ";
res += field.getName() + " = " + field.getValueString();
}
return res;
}
template <typename Traits_>
bool operator==(const BaseSettings<Traits_> & left, const BaseSettings<Traits_> & right)
{
auto l = left.begin();
for (const auto & r : right)
{
if ((l == left.end()) || (*l != r))
return false;
++l;
}
return l == left.end();
}
template <typename Traits_>
bool operator!=(const BaseSettings<Traits_> & left, const BaseSettings<Traits_> & right)
{
return !(left == right);
}
template <typename Traits_>
SettingFieldCustom & BaseSettings<Traits_>::getCustomSetting(const std::string_view & name)
{
if constexpr (Traits::allow_custom_settings)
{
auto it = custom_settings_map.find(name);
if (it == custom_settings_map.end())
{
auto new_name = std::make_shared<String>(name);
it = custom_settings_map.emplace(*new_name, std::make_pair(new_name, SettingFieldCustom{})).first;
}
return it->second.second;
}
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename Traits_>
const SettingFieldCustom & BaseSettings<Traits_>::getCustomSetting(const std::string_view & name) const
{
if constexpr (Traits::allow_custom_settings)
{
auto it = custom_settings_map.find(name);
if (it != custom_settings_map.end())
return it->second.second;
}
BaseSettingsHelpers::throwSettingNotFound(name);
}
template <typename Traits_>
const SettingFieldCustom * BaseSettings<Traits_>::tryGetCustomSetting(const std::string_view & name) const
{
if constexpr (Traits::allow_custom_settings)
{
auto it = custom_settings_map.find(name);
if (it != custom_settings_map.end())
return &it->second.second;
}
return nullptr;
}
template <typename Traits_>
BaseSettings<Traits_>::Iterator::Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_)
: settings(&settings_), accessor(&accessor_), skip_flags(skip_flags_)
{
if (skip_flags == SKIP_ALL)
{
index = accessor->size();
if constexpr (Traits::allow_custom_settings)
custom_settings_iterator = settings->custom_settings_map.end();
return;
}
if (skip_flags & SKIP_CHANGED)
{
skip_flags = static_cast<SkipFlags>(skip_flags | SKIP_CUSTOM);
if (skip_flags & SKIP_UNCHANGED)
skip_flags = static_cast<SkipFlags>(skip_flags | SKIP_BUILTIN);
}
if (skip_flags & SKIP_BUILTIN)
index = accessor->size();
else
index = 0;
if constexpr (Traits::allow_custom_settings)
{
if (skip_flags & SKIP_CUSTOM)
custom_settings_iterator = settings->custom_settings_map.end();
else
custom_settings_iterator = settings->custom_settings_map.begin();
}
doSkip();
}
template <typename Traits_>
typename BaseSettings<Traits_>::Iterator & BaseSettings<Traits_>::Iterator::operator++()
{
if (index != accessor->size())
++index;
else
{
if constexpr (Traits::allow_custom_settings)
++custom_settings_iterator;
}
doSkip();
return *this;
}
template <typename Traits_>
typename BaseSettings<Traits_>::Iterator BaseSettings<Traits_>::Iterator::operator++(int)
{
auto res = *this;
++*this;
return res;
}
template <typename Traits_>
typename BaseSettings<Traits_>::SettingFieldRef BaseSettings<Traits_>::Iterator::operator*() const
{
if constexpr (Traits::allow_custom_settings)
{
if (index == accessor->size())
return {custom_settings_iterator->second};
}
return {*settings, *accessor, index};
}
template <typename Traits_>
void BaseSettings<Traits_>::Iterator::doSkip()
{
if (skip_flags & SKIP_CHANGED)
{
while ((index != accessor->size()) && accessor->isValueChanged(*settings, index))
++index;
}
else if (skip_flags & SKIP_UNCHANGED)
{
while ((index != accessor->size()) && !accessor->isValueChanged(*settings, index))
++index;
}
}
template <typename Traits_>
bool BaseSettings<Traits_>::Iterator::operator ==(const typename BaseSettings<Traits_>::Iterator & other) const
{
if constexpr (Traits_::allow_custom_settings)
{
if (custom_settings_iterator != other.custom_settings_iterator)
return false;
}
return ((index == other.index) && (settings == other.settings));
}
template <typename Traits_>
BaseSettings<Traits_>::SettingFieldRef::SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_)
{
if constexpr (Traits_::allow_custom_settings)
custom_setting = &custom_setting_;
}
template <typename Traits_>
const String & BaseSettings<Traits_>::SettingFieldRef::getName() const
{
if constexpr (Traits::allow_custom_settings)
{
if (custom_setting)
return *custom_setting->first;
}
return accessor->getName(index);
}
template <typename Traits_>
Field BaseSettings<Traits_>::SettingFieldRef::getValue() const
{
if constexpr (Traits::allow_custom_settings)
{
if (custom_setting)
return static_cast<Field>(custom_setting->second);
}
return accessor->getValue(*data, index);
}
template <typename Traits_>
String BaseSettings<Traits_>::SettingFieldRef::getValueString() const
{
if constexpr (Traits::allow_custom_settings)
{
if (custom_setting)
return custom_setting->second.toString();
}
return accessor->getValueString(*data, index);
}
template <typename Traits_>
bool BaseSettings<Traits_>::SettingFieldRef::isValueChanged() const
{
if constexpr (Traits::allow_custom_settings)
{
if (custom_setting)
return true;
}
return accessor->isValueChanged(*data, index);
}
template <typename Traits_>
const char * BaseSettings<Traits_>::SettingFieldRef::getTypeName() const
{
if constexpr (Traits::allow_custom_settings)
{
if (custom_setting)
return "Custom";
}
return accessor->getTypeName(index);
}
template <typename Traits_>
const char * BaseSettings<Traits_>::SettingFieldRef::getDescription() const
{
if constexpr (Traits::allow_custom_settings)
{
if (custom_setting)
return "Custom";
}
return accessor->getDescription(index);
}
template <typename Traits_>
bool BaseSettings<Traits_>::SettingFieldRef::isCustom() const
{
if constexpr (Traits::allow_custom_settings)
return custom_setting != nullptr;
else
return false;
}
#define DECLARE_SETTINGS_TRAITS(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO) \
DECLARE_SETTINGS_TRAITS_COMMON(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO, 0)
#define DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO) \
DECLARE_SETTINGS_TRAITS_COMMON(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO, 1)
#define DECLARE_SETTINGS_TRAITS_COMMON(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO, ALLOW_CUSTOM_SETTINGS) \
struct SETTINGS_TRAITS_NAME \
{ \
struct Data \
{ \
LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_TRAITS_) \
}; \
\
class Accessor \
{ \
public: \
static const Accessor & instance(); \
size_t size() const { return field_infos.size(); } \
size_t find(const std::string_view & name) const; \
const String & getName(size_t index) const { return field_infos[index].name; } \
const char * getTypeName(size_t index) const { return field_infos[index].type; } \
const char * getDescription(size_t index) const { return field_infos[index].description; } \
bool isImportant(size_t index) const { return field_infos[index].is_important; } \
Field castValueUtil(size_t index, const Field & value) const { return field_infos[index].cast_value_util_function(value); } \
String valueToStringUtil(size_t index, const Field & value) const { return field_infos[index].value_to_string_util_function(value); } \
Field stringToValueUtil(size_t index, const String & str) const { return field_infos[index].string_to_value_util_function(str); } \
void setValue(Data & data, size_t index, const Field & value) const { return field_infos[index].set_value_function(data, value); } \
Field getValue(const Data & data, size_t index) const { return field_infos[index].get_value_function(data); } \
void setValueString(Data & data, size_t index, const String & str) const { return field_infos[index].set_value_string_function(data, str); } \
String getValueString(const Data & data, size_t index) const { return field_infos[index].get_value_string_function(data); } \
bool isValueChanged(const Data & data, size_t index) const { return field_infos[index].is_value_changed_function(data); } \
void resetValueToDefault(Data & data, size_t index) const { return field_infos[index].reset_value_to_default_function(data); } \
void writeBinary(const Data & data, size_t index, WriteBuffer & out) const { return field_infos[index].write_binary_function(data, out); } \
void readBinary(Data & data, size_t index, ReadBuffer & in) const { return field_infos[index].read_binary_function(data, in); } \
\
private: \
Accessor(); \
struct FieldInfo \
{ \
String name; \
const char * type; \
const char * description; \
bool is_important; \
Field (*cast_value_util_function)(const Field &); \
String (*value_to_string_util_function)(const Field &); \
Field (*string_to_value_util_function)(const String &); \
void (*set_value_function)(Data &, const Field &) ; \
Field (*get_value_function)(const Data &) ; \
void (*set_value_string_function)(Data &, const String &) ; \
String (*get_value_string_function)(const Data &) ; \
bool (*is_value_changed_function)(const Data &); \
void (*reset_value_to_default_function)(Data &) ; \
void (*write_binary_function)(const Data &, WriteBuffer &) ; \
void (*read_binary_function)(Data &, ReadBuffer &) ; \
}; \
std::vector<FieldInfo> field_infos; \
std::unordered_map<std::string_view, size_t> name_to_index_map; \
}; \
static constexpr bool allow_custom_settings = ALLOW_CUSTOM_SETTINGS; \
};
#define DECLARE_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
SettingField##TYPE NAME {DEFAULT};
#define IMPLEMENT_SETTINGS_TRAITS(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO) \
const SETTINGS_TRAITS_NAME::Accessor & SETTINGS_TRAITS_NAME::Accessor::instance() \
{ \
static const Accessor the_instance = [] \
{ \
Accessor res; \
constexpr int IMPORTANT = 1; \
UNUSED(IMPORTANT); \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_TRAITS_) \
for (size_t i : ext::range(res.field_infos.size())) \
{ \
const auto & info = res.field_infos[i]; \
res.name_to_index_map.emplace(info.name, i); \
} \
return res; \
}(); \
return the_instance; \
} \
\
SETTINGS_TRAITS_NAME::Accessor::Accessor() {} \
\
size_t SETTINGS_TRAITS_NAME::Accessor::find(const std::string_view & name) const \
{ \
auto it = name_to_index_map.find(name); \
if (it != name_to_index_map.end()) \
return it->second; \
return static_cast<size_t>(-1); \
} \
\
template class BaseSettings<SETTINGS_TRAITS_NAME>;
//-V:IMPLEMENT_SETTINGS:501
#define IMPLEMENT_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
res.field_infos.emplace_back( \
FieldInfo{#NAME, #TYPE, DESCRIPTION, FLAGS & IMPORTANT, \
[](const Field & value) -> Field { return static_cast<Field>(SettingField##TYPE{value}); }, \
[](const Field & value) -> String { return SettingField##TYPE{value}.toString(); }, \
[](const String & str) -> Field { SettingField##TYPE temp; temp.parseFromString(str); return static_cast<Field>(temp); }, \
[](Data & data, const Field & value) { data.NAME = value; }, \
[](const Data & data) -> Field { return static_cast<Field>(data.NAME); }, \
[](Data & data, const String & str) { data.NAME.parseFromString(str); }, \
[](const Data & data) -> String { return data.NAME.toString(); }, \
[](const Data & data) -> bool { return data.NAME.changed; }, \
[](Data & data) { data.NAME = SettingField##TYPE{DEFAULT}; }, \
[](const Data & data, WriteBuffer & out) { data.NAME.writeBinary(out); }, \
[](Data & data, ReadBuffer & in) { data.NAME.readBinary(in); } \
});
}

View File

@ -2,7 +2,8 @@
#include <IO/WriteBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/readDecimalText.h>
#include <Core/Field.h>
#include <Core/DecimalComparison.h>
#include <Common/FieldVisitors.h>
@ -10,326 +11,515 @@
namespace DB
{
void readBinary(Array & x, ReadBuffer & buf)
namespace ErrorCodes
{
extern const int CANNOT_RESTORE_FROM_FIELD_DUMP;
extern const int DECIMAL_OVERFLOW;
}
void readBinary(Array & x, ReadBuffer & buf)
{
size_t size;
UInt8 type;
DB::readBinary(type, buf);
DB::readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
{
switch (type)
{
case Field::Types::Null:
{
x.push_back(DB::Field());
break;
}
case Field::Types::UInt64:
{
UInt64 value;
DB::readVarUInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::UInt128:
{
UInt128 value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Int64:
{
Int64 value;
DB::readVarInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::Float64:
{
Float64 value;
DB::readFloatBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::String:
{
std::string value;
DB::readStringBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Array:
{
Array value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Tuple:
{
Tuple value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::AggregateFunctionState:
{
AggregateFunctionStateData value;
DB::readStringBinary(value.name, buf);
DB::readStringBinary(value.data, buf);
x.push_back(value);
break;
}
}
}
}
void writeBinary(const Array & x, WriteBuffer & buf)
{
UInt8 type = Field::Types::Null;
size_t size = x.size();
if (size)
type = x.front().getType();
DB::writeBinary(type, buf);
DB::writeBinary(size, buf);
for (const auto & elem : x)
{
switch (type)
{
case Field::Types::Null: break;
case Field::Types::UInt64:
{
DB::writeVarUInt(get<UInt64>(elem), buf);
break;
}
case Field::Types::UInt128:
{
DB::writeBinary(get<UInt128>(elem), buf);
break;
}
case Field::Types::Int64:
{
DB::writeVarInt(get<Int64>(elem), buf);
break;
}
case Field::Types::Float64:
{
DB::writeFloatBinary(get<Float64>(elem), buf);
break;
}
case Field::Types::String:
{
DB::writeStringBinary(get<std::string>(elem), buf);
break;
}
case Field::Types::Array:
{
DB::writeBinary(get<Array>(elem), buf);
break;
}
case Field::Types::Tuple:
{
DB::writeBinary(get<Tuple>(elem), buf);
break;
}
case Field::Types::AggregateFunctionState:
{
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().name, buf);
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().data, buf);
break;
}
}
}
}
void writeText(const Array & x, WriteBuffer & buf)
{
DB::String res = applyVisitor(DB::FieldVisitorToString(), DB::Field(x));
buf.write(res.data(), res.size());
}
void readBinary(Tuple & x, ReadBuffer & buf)
{
size_t size;
DB::readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
{
size_t size;
UInt8 type;
DB::readBinary(type, buf);
DB::readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
switch (type)
{
switch (type)
case Field::Types::Null:
{
case Field::Types::Null:
{
x.push_back(DB::Field());
break;
}
case Field::Types::UInt64:
{
UInt64 value;
DB::readVarUInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::UInt128:
{
UInt128 value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Int64:
{
Int64 value;
DB::readVarInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::Float64:
{
Float64 value;
DB::readFloatBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::String:
{
std::string value;
DB::readStringBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Array:
{
Array value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Tuple:
{
Tuple value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::AggregateFunctionState:
{
AggregateFunctionStateData value;
DB::readStringBinary(value.name, buf);
DB::readStringBinary(value.data, buf);
x.push_back(value);
break;
}
x.push_back(DB::Field());
break;
}
case Field::Types::UInt64:
{
UInt64 value;
DB::readVarUInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::UInt128:
{
UInt128 value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Int64:
{
Int64 value;
DB::readVarInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::Float64:
{
Float64 value;
DB::readFloatBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::String:
{
std::string value;
DB::readStringBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Array:
{
Array value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Tuple:
{
Tuple value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::AggregateFunctionState:
{
AggregateFunctionStateData value;
DB::readStringBinary(value.name, buf);
DB::readStringBinary(value.data, buf);
x.push_back(value);
break;
}
}
}
void writeBinary(const Array & x, WriteBuffer & buf)
{
UInt8 type = Field::Types::Null;
size_t size = x.size();
if (size)
type = x.front().getType();
DB::writeBinary(type, buf);
DB::writeBinary(size, buf);
for (const auto & elem : x)
{
switch (type)
{
case Field::Types::Null: break;
case Field::Types::UInt64:
{
DB::writeVarUInt(get<UInt64>(elem), buf);
break;
}
case Field::Types::UInt128:
{
DB::writeBinary(get<UInt128>(elem), buf);
break;
}
case Field::Types::Int64:
{
DB::writeVarInt(get<Int64>(elem), buf);
break;
}
case Field::Types::Float64:
{
DB::writeFloatBinary(get<Float64>(elem), buf);
break;
}
case Field::Types::String:
{
DB::writeStringBinary(get<std::string>(elem), buf);
break;
}
case Field::Types::Array:
{
DB::writeBinary(get<Array>(elem), buf);
break;
}
case Field::Types::Tuple:
{
DB::writeBinary(get<Tuple>(elem), buf);
break;
}
case Field::Types::AggregateFunctionState:
{
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().name, buf);
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().data, buf);
break;
}
}
}
}
void writeText(const Array & x, WriteBuffer & buf)
{
DB::String res = applyVisitor(DB::FieldVisitorToString(), DB::Field(x));
buf.write(res.data(), res.size());
}
void readBinary(Tuple & x, ReadBuffer & buf)
{
size_t size;
DB::readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
{
UInt8 type;
DB::readBinary(type, buf);
switch (type)
{
case Field::Types::Null:
{
x.push_back(DB::Field());
break;
}
case Field::Types::UInt64:
{
UInt64 value;
DB::readVarUInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::UInt128:
{
UInt128 value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Int64:
{
Int64 value;
DB::readVarInt(value, buf);
x.push_back(value);
break;
}
case Field::Types::Float64:
{
Float64 value;
DB::readFloatBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::String:
{
std::string value;
DB::readStringBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Array:
{
Array value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::Tuple:
{
Tuple value;
DB::readBinary(value, buf);
x.push_back(value);
break;
}
case Field::Types::AggregateFunctionState:
{
AggregateFunctionStateData value;
DB::readStringBinary(value.name, buf);
DB::readStringBinary(value.data, buf);
x.push_back(value);
break;
}
}
}
}
void writeBinary(const Tuple & x, WriteBuffer & buf)
{
const size_t size = x.size();
DB::writeBinary(size, buf);
for (const auto & elem : x)
{
const UInt8 type = elem.getType();
DB::writeBinary(type, buf);
switch (type)
{
case Field::Types::Null: break;
case Field::Types::UInt64:
{
DB::writeVarUInt(get<UInt64>(elem), buf);
break;
}
case Field::Types::UInt128:
{
DB::writeBinary(get<UInt128>(elem), buf);
break;
}
case Field::Types::Int64:
{
DB::writeVarInt(get<Int64>(elem), buf);
break;
}
case Field::Types::Float64:
{
DB::writeFloatBinary(get<Float64>(elem), buf);
break;
}
case Field::Types::String:
{
DB::writeStringBinary(get<std::string>(elem), buf);
break;
}
case Field::Types::Array:
{
DB::writeBinary(get<Array>(elem), buf);
break;
}
case Field::Types::Tuple:
{
DB::writeBinary(get<Tuple>(elem), buf);
break;
}
case Field::Types::AggregateFunctionState:
{
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().name, buf);
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().data, buf);
break;
}
}
}
}
void writeText(const Tuple & x, WriteBuffer & buf)
{
writeFieldText(DB::Field(x), buf);
}
void writeFieldText(const Field & x, WriteBuffer & buf)
{
DB::String res = Field::dispatch(DB::FieldVisitorToString(), x);
buf.write(res.data(), res.size());
}
template <typename T>
static bool decEqual(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, EqualsOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <typename T>
static bool decLess(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, LessOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <typename T>
static bool decLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, LessOrEqualsOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <> bool decimalEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); }
template <> bool decimalLess(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
template <> bool decimalEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); }
template <> bool decimalLess(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
template <> bool decimalEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); }
template <> bool decimalLess(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
}
void writeBinary(const Tuple & x, WriteBuffer & buf)
{
const size_t size = x.size();
DB::writeBinary(size, buf);
for (const auto & elem : x)
{
const UInt8 type = elem.getType();
DB::writeBinary(type, buf);
switch (type)
{
case Field::Types::Null: break;
case Field::Types::UInt64:
{
DB::writeVarUInt(get<UInt64>(elem), buf);
break;
}
case Field::Types::UInt128:
{
DB::writeBinary(get<UInt128>(elem), buf);
break;
}
case Field::Types::Int64:
{
DB::writeVarInt(get<Int64>(elem), buf);
break;
}
case Field::Types::Float64:
{
DB::writeFloatBinary(get<Float64>(elem), buf);
break;
}
case Field::Types::String:
{
DB::writeStringBinary(get<std::string>(elem), buf);
break;
}
case Field::Types::Array:
{
DB::writeBinary(get<Array>(elem), buf);
break;
}
case Field::Types::Tuple:
{
DB::writeBinary(get<Tuple>(elem), buf);
break;
}
case Field::Types::AggregateFunctionState:
{
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().name, buf);
DB::writeStringBinary(elem.get<AggregateFunctionStateData>().data, buf);
break;
}
}
}
}
void writeText(const Tuple & x, WriteBuffer & buf)
{
writeFieldText(DB::Field(x), buf);
}
template <typename T>
void readQuoted(DecimalField<T> & x, ReadBuffer & buf)
{
assertChar('\'', buf);
T value;
UInt32 scale;
int32_t exponent;
uint32_t max_digits = static_cast<uint32_t>(-1);
readDigits<true>(buf, value, max_digits, exponent, true);
if (exponent > 0)
{
scale = 0;
if (common::mulOverflow(value.value, T::getScaleMultiplier(exponent), value.value))
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
}
else
scale = -exponent;
assertChar('\'', buf);
x = DecimalField<T>{value, scale};
}
template void readQuoted<Decimal32>(DecimalField<Decimal32> & x, ReadBuffer & buf);
template void readQuoted<Decimal64>(DecimalField<Decimal64> & x, ReadBuffer & buf);
template void readQuoted<Decimal128>(DecimalField<Decimal128> & x, ReadBuffer & buf);
void writeFieldText(const Field & x, WriteBuffer & buf)
{
DB::String res = Field::dispatch(DB::FieldVisitorToString(), x);
buf.write(res.data(), res.size());
}
String Field::dump() const
{
return applyVisitor(DB::FieldVisitorDump(), *this);
}
Field Field::restoreFromDump(const std::string_view & dump_)
{
auto show_error = [&dump_]
{
throw Exception("Couldn't restore Field from dump: " + String{dump_}, ErrorCodes::CANNOT_RESTORE_FROM_FIELD_DUMP);
};
std::string_view dump = dump_;
trim(dump);
if (dump == "NULL")
return {};
std::string_view prefix = std::string_view{"Int64_"};
if (dump.starts_with(prefix))
{
Int64 value = parseFromString<Int64>(dump.substr(prefix.length()));
return value;
}
prefix = std::string_view{"UInt64_"};
if (dump.starts_with(prefix))
{
UInt64 value = parseFromString<UInt64>(dump.substr(prefix.length()));
return value;
}
prefix = std::string_view{"Float64_"};
if (dump.starts_with(prefix))
{
Float64 value = parseFromString<Float64>(dump.substr(prefix.length()));
return value;
}
prefix = std::string_view{"Decimal32_"};
if (dump_.starts_with(prefix))
{
DecimalField<Decimal32> decimal;
ReadBufferFromString buf{dump.substr(prefix.length())};
readQuoted(decimal, buf);
return decimal;
}
prefix = std::string_view{"Decimal64_"};
if (dump_.starts_with(prefix))
{
DecimalField<Decimal64> decimal;
ReadBufferFromString buf{dump.substr(prefix.length())};
readQuoted(decimal, buf);
return decimal;
}
prefix = std::string_view{"Decimal128_"};
if (dump_.starts_with(prefix))
{
DecimalField<Decimal128> decimal;
ReadBufferFromString buf{dump.substr(prefix.length())};
readQuoted(decimal, buf);
return decimal;
}
prefix = std::string_view{"UUID_"};
if (dump.starts_with(prefix))
{
UUID uuid;
ReadBufferFromString buf{dump.substr(prefix.length())};
readQuoted(uuid, buf);
return uuid;
}
if (dump.starts_with("\'"))
{
String str;
ReadBufferFromString buf{dump};
readQuoted(str, buf);
return str;
}
prefix = std::string_view{"Array_["};
if (dump.starts_with(prefix))
{
std::string_view tail = dump.substr(prefix.length());
trimLeft(tail);
Array array;
while (tail != "]")
{
size_t separator = tail.find_first_of(",]");
if (separator == std::string_view::npos)
show_error();
bool comma = (tail[separator] == ',');
std::string_view element = tail.substr(0, separator);
tail.remove_prefix(separator);
if (comma)
tail.remove_prefix(1);
trimLeft(tail);
if (!comma && tail != "]")
show_error();
array.push_back(Field::restoreFromDump(element));
}
return array;
}
prefix = std::string_view{"Tuple_("};
if (dump.starts_with(prefix))
{
std::string_view tail = dump.substr(prefix.length());
trimLeft(tail);
Tuple tuple;
while (tail != ")")
{
size_t separator = tail.find_first_of(",)");
if (separator == std::string_view::npos)
show_error();
bool comma = (tail[separator] == ',');
std::string_view element = tail.substr(0, separator);
tail.remove_prefix(separator);
if (comma)
tail.remove_prefix(1);
trimLeft(tail);
if (!comma && tail != ")")
show_error();
tuple.push_back(Field::restoreFromDump(element));
}
return tuple;
}
prefix = std::string_view{"AggregateFunctionState_("};
if (dump.starts_with(prefix))
{
std::string_view after_prefix = dump.substr(prefix.length());
size_t comma = after_prefix.find(',');
size_t end = after_prefix.find(')', comma + 1);
if ((comma == std::string_view::npos) || (end != after_prefix.length() - 1))
show_error();
std::string_view name_view = after_prefix.substr(0, comma);
std::string_view data_view = after_prefix.substr(comma + 1, end - comma - 1);
trim(name_view);
trim(data_view);
ReadBufferFromString name_buf{name_view};
ReadBufferFromString data_buf{data_view};
AggregateFunctionStateData res;
readQuotedString(res.name, name_buf);
readQuotedString(res.data, data_buf);
return res;
}
show_error();
__builtin_unreachable();
}
template <typename T>
static bool decEqual(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, EqualsOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <typename T>
static bool decLess(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, LessOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <typename T>
static bool decLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale)
{
using Comparator = DecimalComparison<T, T, LessOrEqualsOp>;
return Comparator::compare(x, y, x_scale, y_scale);
}
template <> bool decimalEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); }
template <> bool decimalLess(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
template <> bool decimalEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); }
template <> bool decimalLess(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
template <> bool decimalEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); }
template <> bool decimalLess(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
}

View File

@ -95,7 +95,7 @@ template <typename T>
class DecimalField
{
public:
DecimalField(T value, UInt32 scale_)
DecimalField(T value = 0, UInt32 scale_ = 0)
: dec(value),
scale(scale_)
{}
@ -187,6 +187,7 @@ template <> struct NearestFieldTypeImpl<DecimalField<Decimal128>> { using Type =
template <> struct NearestFieldTypeImpl<Float32> { using Type = Float64; };
template <> struct NearestFieldTypeImpl<Float64> { using Type = Float64; };
template <> struct NearestFieldTypeImpl<const char *> { using Type = String; };
template <> struct NearestFieldTypeImpl<std::string_view> { using Type = String; };
template <> struct NearestFieldTypeImpl<String> { using Type = String; };
template <> struct NearestFieldTypeImpl<Array> { using Type = Array; };
template <> struct NearestFieldTypeImpl<Tuple> { using Type = Tuple; };
@ -272,7 +273,10 @@ public:
template <typename T> struct TypeToEnum;
template <Types::Which which> struct EnumToType;
static bool IsDecimal(Types::Which which) { return which >= Types::Decimal32 && which <= Types::Decimal128; }
/// Templates to avoid ambiguity.
template <typename T, typename Z = void *>
using enable_if_not_field_or_stringlike_t = std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field> && !std::is_same_v<NearestFieldType<std::decay_t<T>>, String>, Z>;
Field()
: which(Types::Null)
@ -293,23 +297,20 @@ public:
}
template <typename T>
Field(T && rhs, std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field>, void *> = nullptr);
Field(T && rhs, enable_if_not_field_or_stringlike_t<T> = nullptr);
/// Create a string inplace.
Field(const std::string_view & str) { create(str.data(), str.size()); }
Field(const String & str) { create(std::string_view{str}); }
Field(String && str) { create(std::move(str)); }
Field(const char * str) { create(std::string_view{str}); }
template <typename CharT>
Field(const CharT * data, size_t size)
{
create(data, size);
}
/// NOTE In case when field already has string type, more direct assign is possible.
template <typename CharT>
void assignString(const CharT * data, size_t size)
{
destroy();
create(data, size);
}
Field & operator= (const Field & rhs)
{
if (this != &rhs)
@ -340,9 +341,19 @@ public:
return *this;
}
/// Allows expressions like
/// Field f = 1;
/// Things to note:
/// 1. float <--> int needs explicit cast
/// 2. customized types needs explicit cast
template <typename T>
std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field>, Field &>
operator= (T && rhs);
enable_if_not_field_or_stringlike_t<T, Field> &
operator=(T && rhs);
Field & operator =(const std::string_view & str);
Field & operator =(const String & str) { return *this = std::string_view{str}; }
Field & operator =(String && str);
Field & operator =(const char * str) { return *this = std::string_view{str}; }
~Field()
{
@ -410,7 +421,6 @@ public:
return get<T>();
}
bool operator< (const Field & rhs) const
{
if (which < rhs.which)
@ -554,6 +564,8 @@ public:
return f(null);
}
String dump() const;
static Field restoreFromDump(const std::string_view & dump_);
private:
std::aligned_union_t<DBMS_MIN_FIELD_SIZE - sizeof(Types::Which),
@ -590,6 +602,20 @@ private:
*ptr = std::forward<T>(x);
}
template <typename CharT>
std::enable_if_t<sizeof(CharT) == 1> assignString(const CharT * data, size_t size)
{
assert(which == Types::String);
String * ptr = reinterpret_cast<String *>(&storage);
ptr->assign(reinterpret_cast<const char *>(data), size);
}
void assignString(String && str)
{
assert(which == Types::String);
String * ptr = reinterpret_cast<String *>(&storage);
ptr->assign(std::move(str));
}
void create(const Field & x)
{
@ -618,6 +644,12 @@ private:
which = Types::String;
}
void create(String && str)
{
new (&storage) String(std::move(str));
which = Types::String;
}
ALWAYS_INLINE void destroy()
{
if (which < Types::MIN_NON_POD)
@ -753,23 +785,16 @@ decltype(auto) castToNearestFieldType(T && x)
return U(x);
}
/// This (rather tricky) code is to avoid ambiguity in expressions like
/// Field f = 1;
/// instead of
/// Field f = Int64(1);
/// Things to note:
/// 1. float <--> int needs explicit cast
/// 2. customized types needs explicit cast
template <typename T>
Field::Field(T && rhs, std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field>, void *>)
Field::Field(T && rhs, enable_if_not_field_or_stringlike_t<T>)
{
auto && val = castToNearestFieldType(std::forward<T>(rhs));
createConcrete(std::forward<decltype(val)>(val));
}
template <typename T>
std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field>, Field &>
Field::operator= (T && rhs)
Field::enable_if_not_field_or_stringlike_t<T, Field> &
Field::operator=(T && rhs)
{
auto && val = castToNearestFieldType(std::forward<T>(rhs));
using U = decltype(val);
@ -780,11 +805,34 @@ Field::operator= (T && rhs)
}
else
assignConcrete(std::forward<U>(val));
return *this;
}
inline Field & Field::operator=(const std::string_view & str)
{
if (which != Types::String)
{
destroy();
create(str.data(), str.size());
}
else
assignString(str.data(), str.size());
return *this;
}
inline Field & Field::operator=(String && str)
{
if (which != Types::String)
{
destroy();
create(std::move(str));
}
else
assignString(std::move(str));
return *this;
}
class ReadBuffer;
class WriteBuffer;
@ -811,6 +859,9 @@ void writeBinary(const Tuple & x, WriteBuffer & buf);
void writeText(const Tuple & x, WriteBuffer & buf);
template <typename T>
void readQuoted(DecimalField<T> & x, ReadBuffer & buf);
void writeFieldText(const Field & x, WriteBuffer & buf);
[[noreturn]] inline void writeQuoted(const Tuple &, WriteBuffer &) { throw Exception("Cannot write Tuple quoted.", ErrorCodes::NOT_IMPLEMENTED); }

View File

@ -5,7 +5,6 @@
#include <Common/typeid_cast.h>
#include <string.h>
#include <boost/program_options/options_description.hpp>
#include <Core/SettingsCollectionImpl.h>
namespace DB
{
@ -17,7 +16,7 @@ namespace ErrorCodes
}
IMPLEMENT_SETTINGS_COLLECTION(Settings, LIST_OF_SETTINGS)
IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS)
/** Set the settings from the profile (in the server configuration, many settings can be listed in one profile).
@ -64,27 +63,24 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu
auto * column_names = (column_names_) ? &typeid_cast<ColumnArray &>(*column_names_) : nullptr;
auto * column_values = (column_values_) ? &typeid_cast<ColumnArray &>(*column_values_) : nullptr;
size_t size = 0;
size_t count = 0;
for (const auto & setting : *this)
for (auto setting : all(changed_only ? SKIP_UNCHANGED : SKIP_NONE))
{
if (!changed_only || setting.isChanged())
if (column_names)
{
if (column_names)
{
StringRef name = setting.getName();
column_names->getData().insertData(name.data, name.size);
}
if (column_values)
column_values->getData().insert(setting.getValueAsString());
++size;
auto name = setting.getName();
column_names->getData().insertData(name.data(), name.size());
}
if (column_values)
column_values->getData().insert(setting.getValueString());
++count;
}
if (column_names)
{
auto & offsets = column_names->getOffsets();
offsets.push_back(offsets.back() + size);
offsets.push_back(offsets.back() + count);
}
/// Nested columns case
@ -93,20 +89,21 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu
if (column_values && !the_same_offsets)
{
auto & offsets = column_values->getOffsets();
offsets.push_back(offsets.back() + size);
offsets.push_back(offsets.back() + count);
}
}
void Settings::addProgramOptions(boost::program_options::options_description & options)
{
for (size_t index = 0; index != Settings::size(); ++index)
for (auto field : all())
{
const std::string_view name = field.getName();
auto on_program_option
= boost::function1<void, const std::string &>([this, index](const std::string & value) { set(index, value); });
= boost::function1<void, const std::string &>([this, name](const std::string & value) { set(name, value); });
options.add(boost::shared_ptr<boost::program_options::option_description>(new boost::program_options::option_description(
Settings::getName(index).data,
name.data(),
boost::program_options::value<std::string>()->composing()->notifier(on_program_option),
Settings::getDescription(index).data)));
field.getDescription())));
}
}
}

View File

@ -1,238 +1,223 @@
#pragma once
#include <Core/SettingsCollection.h>
#include <Core/BaseSettings.h>
#include <Core/SettingsEnums.h>
#include <Core/Defines.h>
namespace Poco
namespace Poco::Util
{
namespace Util
{
class AbstractConfiguration;
}
class AbstractConfiguration;
}
namespace boost
namespace boost::program_options
{
namespace program_options
{
class options_description;
}
class options_description;
}
namespace DB
{
class IColumn;
/** Settings of query execution.
* These settings go to users.xml.
/** List of settings: type, name, default value, description, flags
*
* This looks rather unconvenient. It is done that way to avoid repeating settings in different places.
* Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field,
* but we are not going to do it, because settings is used everywhere as static struct fields.
*
* `flags` can be either 0 or IMPORTANT.
* A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions.
*/
struct Settings : public SettingsCollection<Settings>
{
/// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14.
/// http://en.cppreference.com/w/cpp/language/aggregate_initialization
Settings() {}
/** List of settings: type, name, default value, description, flags
*
* This looks rather unconvenient. It is done that way to avoid repeating settings in different places.
* Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field,
* but we are not going to do it, because settings is used everywhere as static struct fields.
*
* `flags` can be either 0 or IMPORTANT.
* A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions.
*/
#define COMMON_SETTINGS(M) \
M(SettingUInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \
M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \
M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \
M(SettingUInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \
M(SettingUInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \
M(SettingUInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \
M(SettingUInt64, min_insert_block_size_rows_for_materialized_views, 0, "Like min_insert_block_size_rows, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_rows)", 0) \
M(SettingUInt64, min_insert_block_size_bytes_for_materialized_views, 0, "Like min_insert_block_size_bytes, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_bytes)", 0) \
M(SettingUInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, "Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.", 0) \
M(SettingUInt64, max_insert_threads, 0, "The maximum number of threads to execute the INSERT SELECT query. Values 0 or 1 means that INSERT SELECT is not run in parallel. Higher values will lead to higher memory usage. Parallel INSERT SELECT has effect only if the SELECT part is run on parallel, see 'max_threads' setting.", 0) \
M(SettingUInt64, max_final_threads, 16, "The maximum number of threads to read from table with FINAL.", 0) \
M(SettingMaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \
M(SettingMaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \
M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \
M(SettingUInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \
M(SettingUInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \
M(SettingUInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \
M(SettingSeconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \
M(SettingMilliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.", 0) \
M(SettingMilliseconds, connect_timeout_with_failover_secure_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_SECURE_MS, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \
M(SettingSeconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \
M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \
M(SettingSeconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \
M(SettingMilliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \
M(SettingMilliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \
M(SettingMilliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.", 0) \
M(SettingMilliseconds, rabbitmq_max_wait_ms, 5000, "The wait time for reading from RabbitMQ before retry.", 0) \
M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.", 0) \
M(SettingUInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \
M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \
M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \
M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \
M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \
M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \
M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \
M(SettingUInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \
M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \
M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \
M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, kafka streaming, dns cache updates. Only has meaning at server startup.", 0) \
M(SettingUInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \
#define COMMON_SETTINGS(M) \
M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \
M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \
M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \
M(UInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \
M(UInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \
M(UInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \
M(UInt64, min_insert_block_size_rows_for_materialized_views, 0, "Like min_insert_block_size_rows, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_rows)", 0) \
M(UInt64, min_insert_block_size_bytes_for_materialized_views, 0, "Like min_insert_block_size_bytes, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_bytes)", 0) \
M(UInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, "Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.", 0) \
M(UInt64, max_insert_threads, 0, "The maximum number of threads to execute the INSERT SELECT query. Values 0 or 1 means that INSERT SELECT is not run in parallel. Higher values will lead to higher memory usage. Parallel INSERT SELECT has effect only if the SELECT part is run on parallel, see 'max_threads' setting.", 0) \
M(UInt64, max_final_threads, 16, "The maximum number of threads to read from table with FINAL.", 0) \
M(MaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \
M(MaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \
M(UInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \
M(UInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \
M(UInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \
M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \
M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \
M(Milliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.", 0) \
M(Milliseconds, connect_timeout_with_failover_secure_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_SECURE_MS, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \
M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \
M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \
M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \
M(Milliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \
M(Milliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \
M(Milliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.", 0) \
M(Milliseconds, rabbitmq_max_wait_ms, 5000, "The wait time for reading from RabbitMQ before retry.", 0) \
M(UInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.", 0) \
M(UInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \
M(UInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \
M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \
M(UInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \
M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \
M(Bool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \
M(Bool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \
M(UInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \
M(UInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \
M(UInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \
M(UInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, kafka streaming, dns cache updates. Only has meaning at server startup.", 0) \
M(UInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \
\
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \
M(SettingMilliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \
M(Milliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \
M(Milliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \
\
M(SettingBool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.", 0) \
M(Bool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.", 0) \
\
M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \
M(Bool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \
\
M(SettingMilliseconds, insert_in_memory_parts_timeout, 600000, "", 0) \
M(Milliseconds, insert_in_memory_parts_timeout, 600000, "", 0) \
\
M(SettingUInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) \
M(SettingUInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \
M(UInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) \
M(UInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \
\
M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \
M(LoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \
\
M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = any are present.", IMPORTANT) \
M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \
M(TotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = any are present.", IMPORTANT) \
M(Float, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \
\
M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(SettingUInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \
M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \
M(SettingBool, distributed_aggregation_memory_efficient, false, "Is the memory-saving mode of distributed aggregation enabled.", 0) \
M(SettingUInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \
M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \
M(UInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \
M(Bool, distributed_aggregation_memory_efficient, false, "Is the memory-saving mode of distributed aggregation enabled.", 0) \
M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \
\
M(SettingUInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \
M(SettingUInt64, parallel_replicas_count, 0, "", 0) \
M(SettingUInt64, parallel_replica_offset, 0, "", 0) \
M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \
M(UInt64, parallel_replicas_count, 0, "", 0) \
M(UInt64, parallel_replica_offset, 0, "", 0) \
\
M(SettingSpecialSort, special_sort, SpecialSort::NOT_SPECIFIED, "Specifies a sorting algorithm which will be using in ORDER BY query.", 0) \
M(SpecialSort, special_sort, SpecialSort::NOT_SPECIFIED, "Specifies a sorting algorithm which will be using in ORDER BY query.", 0) \
\
M(SettingBool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \
M(Bool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \
\
M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \
M(SettingBool, parallel_distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \
M(SettingBool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avodiing costly aggregation on the initiator server).", 0) \
M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
M(SettingUInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \
M(SettingUInt64, optimize_skip_unused_shards_nesting, 0, "Same as optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \
M(SettingUInt64, force_optimize_skip_unused_shards_nesting, 0, "Same as force_optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \
M(Bool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \
M(Bool, parallel_distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \
M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avodiing costly aggregation on the initiator server).", 0) \
M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
M(UInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \
M(UInt64, optimize_skip_unused_shards_nesting, 0, "Same as optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \
M(UInt64, force_optimize_skip_unused_shards_nesting, 0, "Same as force_optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \
\
M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \
M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \
M(Bool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \
M(UInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \
\
M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \
M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \
M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.", 0) \
M(SettingUInt64, merge_tree_min_bytes_for_seek, 0, "You can skip reading more than that number of bytes at the price of one seek per file.", 0) \
M(SettingUInt64, merge_tree_coarse_index_granularity, 8, "If the index segment can contain the required keys, divide it into as many parts and recursively check them.", 0) \
M(SettingUInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
M(SettingUInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
M(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \
M(UInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \
M(UInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.", 0) \
M(UInt64, merge_tree_min_bytes_for_seek, 0, "You can skip reading more than that number of bytes at the price of one seek per file.", 0) \
M(UInt64, merge_tree_coarse_index_granularity, 8, "If the index segment can contain the required keys, divide it into as many parts and recursively check them.", 0) \
M(UInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
M(UInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \
\
M(SettingUInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \
M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \
\
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \
M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \
\
M(SettingUInt64, min_bytes_to_use_direct_io, 0, "The minimum number of bytes for reading the data with O_DIRECT option during SELECT queries execution. 0 - disabled.", 0) \
M(SettingUInt64, min_bytes_to_use_mmap_io, 0, "The minimum number of bytes for reading the data with mmap option during SELECT queries execution. 0 - disabled.", 0) \
M(UInt64, min_bytes_to_use_direct_io, 0, "The minimum number of bytes for reading the data with O_DIRECT option during SELECT queries execution. 0 - disabled.", 0) \
M(UInt64, min_bytes_to_use_mmap_io, 0, "The minimum number of bytes for reading the data with mmap option during SELECT queries execution. 0 - disabled.", 0) \
\
M(SettingBool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \
M(SettingBool, force_primary_key, 0, "Throw an exception if there is primary key in a table, and it is not used.", 0) \
M(Bool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \
M(Bool, force_primary_key, 0, "Throw an exception if there is primary key in a table, and it is not used.", 0) \
\
M(SettingFloat, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.", 0) \
M(SettingFloat, max_streams_multiplier_for_merge_tables, 5, "Ask more streams when reading from Merge table. Streams will be spread across tables that Merge table will use. This allows more even distribution of work across threads and especially helpful when merged tables differ in size.", 0) \
M(Float, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.", 0) \
M(Float, max_streams_multiplier_for_merge_tables, 5, "Ask more streams when reading from Merge table. Streams will be spread across tables that Merge table will use. This allows more even distribution of work across threads and especially helpful when merged tables differ in size.", 0) \
\
M(SettingString, network_compression_method, "LZ4", "Allows you to select the method of data compression when writing.", 0) \
M(String, network_compression_method, "LZ4", "Allows you to select the method of data compression when writing.", 0) \
\
M(SettingInt64, network_zstd_compression_level, 1, "Allows you to select the level of ZSTD compression.", 0) \
M(Int64, network_zstd_compression_level, 1, "Allows you to select the level of ZSTD compression.", 0) \
\
M(SettingUInt64, priority, 0, "Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities.", 0) \
M(SettingInt64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.", 0) \
M(UInt64, priority, 0, "Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities.", 0) \
M(Int64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.", 0) \
\
M(SettingBool, log_queries, 1, "Log requests and write the log to the system table.", 0) \
M(SettingLogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \
M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \
M(Bool, log_queries, 1, "Log requests and write the log to the system table.", 0) \
M(LogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \
M(UInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \
\
M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \
M(DistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \
\
M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \
M(UInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \
\
M(SettingBool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed", 0) \
M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed", 0) \
\
M(SettingUInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \
M(SettingMilliseconds, insert_quorum_timeout, 600000, "", 0) \
M(SettingUInt64, select_sequential_consistency, 0, "For SELECT queries from the replicated table, throw an exception if the replica does not have a chunk written with the quorum; do not read the parts that have not yet been written with the quorum.", 0) \
M(SettingUInt64, table_function_remote_max_addresses, 1000, "The maximum number of different shards and the maximum number of replicas of one shard in the `remote` function.", 0) \
M(SettingMilliseconds, read_backoff_min_latency_ms, 1000, "Setting to reduce the number of threads in case of slow reads. Pay attention only to reads that took at least that much time.", 0) \
M(SettingUInt64, read_backoff_max_throughput, 1048576, "Settings to reduce the number of threads in case of slow reads. Count events when the read bandwidth is less than that many bytes per second.", 0) \
M(SettingMilliseconds, read_backoff_min_interval_between_events_ms, 1000, "Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time.", 0) \
M(SettingUInt64, read_backoff_min_events, 2, "Settings to reduce the number of threads in case of slow reads. The number of events after which the number of threads will be reduced.", 0) \
M(UInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \
M(Milliseconds, insert_quorum_timeout, 600000, "", 0) \
M(UInt64, select_sequential_consistency, 0, "For SELECT queries from the replicated table, throw an exception if the replica does not have a chunk written with the quorum; do not read the parts that have not yet been written with the quorum.", 0) \
M(UInt64, table_function_remote_max_addresses, 1000, "The maximum number of different shards and the maximum number of replicas of one shard in the `remote` function.", 0) \
M(Milliseconds, read_backoff_min_latency_ms, 1000, "Setting to reduce the number of threads in case of slow reads. Pay attention only to reads that took at least that much time.", 0) \
M(UInt64, read_backoff_max_throughput, 1048576, "Settings to reduce the number of threads in case of slow reads. Count events when the read bandwidth is less than that many bytes per second.", 0) \
M(Milliseconds, read_backoff_min_interval_between_events_ms, 1000, "Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time.", 0) \
M(UInt64, read_backoff_min_events, 2, "Settings to reduce the number of threads in case of slow reads. The number of events after which the number of threads will be reduced.", 0) \
\
M(SettingFloat, memory_tracker_fault_probability, 0., "For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.", 0) \
M(Float, memory_tracker_fault_probability, 0., "For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.", 0) \
\
M(SettingBool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.", 0) \
M(SettingInt64, http_zlib_compression_level, 3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.", 0) \
M(Bool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.", 0) \
M(Int64, http_zlib_compression_level, 3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.", 0) \
\
M(SettingBool, http_native_compression_disable_checksumming_on_decompress, 0, "If you uncompress the POST data from the client compressed by the native format, do not check the checksum.", 0) \
M(Bool, http_native_compression_disable_checksumming_on_decompress, 0, "If you uncompress the POST data from the client compressed by the native format, do not check the checksum.", 0) \
\
M(SettingString, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \
M(String, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \
\
M(SettingBool, add_http_cors_header, false, "Write add http CORS header.", 0) \
M(Bool, add_http_cors_header, false, "Write add http CORS header.", 0) \
\
M(SettingUInt64, max_http_get_redirects, 0, "Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.", 0) \
M(UInt64, max_http_get_redirects, 0, "Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.", 0) \
\
M(SettingBool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \
M(Bool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \
\
M(SettingBool, send_progress_in_http_headers, false, "Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python requests in particular), so it is disabled by default.", 0) \
M(Bool, send_progress_in_http_headers, false, "Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python requests in particular), so it is disabled by default.", 0) \
\
M(SettingUInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \
M(UInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \
\
M(SettingBool, fsync_metadata, 1, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
M(Bool, fsync_metadata, 1, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
\
M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \
M(Bool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \
\
M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \
M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \
M(JoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \
M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \
\
M(SettingUInt64, preferred_block_size_bytes, 1000000, "", 0) \
M(UInt64, preferred_block_size_bytes, 1000000, "", 0) \
\
M(SettingUInt64, max_replica_delay_for_distributed_queries, 300, "If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.", 0) \
M(SettingBool, fallback_to_stale_replicas_for_distributed_queries, 1, "Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. If this setting is enabled, the query will be performed anyway, otherwise the error will be reported.", 0) \
M(SettingUInt64, preferred_max_column_in_block_size_bytes, 0, "Limit on max column size in block while reading. Helps to decrease cache misses count. Should be close to L2 cache size.", 0) \
M(UInt64, max_replica_delay_for_distributed_queries, 300, "If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.", 0) \
M(Bool, fallback_to_stale_replicas_for_distributed_queries, 1, "Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. If this setting is enabled, the query will be performed anyway, otherwise the error will be reported.", 0) \
M(UInt64, preferred_max_column_in_block_size_bytes, 0, "Limit on max column size in block while reading. Helps to decrease cache misses count. Should be close to L2 cache size.", 0) \
\
M(SettingBool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.", 0) \
M(SettingUInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) \
M(SettingInt64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite.", 0) \
M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \
M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \
M(Bool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.", 0) \
M(UInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) \
M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite.", 0) \
M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \
M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \
\
M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \
M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \
M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \
M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \
M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \
M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
M(SettingBool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \
M(SettingBool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \
M(SettingUInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(SettingUInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(SettingBool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \
M(SettingString, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \
M(Bool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \
M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \
M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \
M(Seconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \
M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \
M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \
M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \
M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \
M(UInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \
M(UInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(UInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \
M(String, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \
\
\
/** Limits during query execution are part of the settings. \
@ -242,243 +227,253 @@ struct Settings : public SettingsCollection<Settings>
* Almost all limits apply to each stream individually. \
*/ \
\
M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \
M(SettingUInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \
M(SettingOverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(UInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \
M(UInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \
M(OverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
\
M(SettingUInt64, max_rows_to_group_by, 0, "", 0) \
M(SettingOverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(SettingUInt64, max_bytes_before_external_group_by, 0, "", 0) \
M(UInt64, max_rows_to_group_by, 0, "", 0) \
M(OverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(UInt64, max_bytes_before_external_group_by, 0, "", 0) \
\
M(SettingUInt64, max_rows_to_sort, 0, "", 0) \
M(SettingUInt64, max_bytes_to_sort, 0, "", 0) \
M(SettingOverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(SettingUInt64, max_bytes_before_external_sort, 0, "", 0) \
M(SettingUInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \
M(UInt64, max_rows_to_sort, 0, "", 0) \
M(UInt64, max_bytes_to_sort, 0, "", 0) \
M(OverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(UInt64, max_bytes_before_external_sort, 0, "", 0) \
M(UInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \
\
M(SettingUInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.", 0) \
M(SettingUInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.", 0) \
M(SettingOverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(UInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.", 0) \
M(UInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.", 0) \
M(OverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
\
/* TODO: Check also when merging and finalizing aggregate functions. */ \
M(SettingSeconds, max_execution_time, 0, "", 0) \
M(SettingOverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(Seconds, max_execution_time, 0, "", 0) \
M(OverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
\
M(SettingUInt64, min_execution_speed, 0, "Minimum number of execution rows per second.", 0) \
M(SettingUInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \
M(SettingUInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.", 0) \
M(SettingUInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.", 0) \
M(SettingSeconds, timeout_before_checking_execution_speed, 10, "Check that the speed is not too low after the specified time has elapsed.", 0) \
M(UInt64, min_execution_speed, 0, "Minimum number of execution rows per second.", 0) \
M(UInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \
M(UInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.", 0) \
M(UInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.", 0) \
M(Seconds, timeout_before_checking_execution_speed, 10, "Check that the speed is not too low after the specified time has elapsed.", 0) \
\
M(SettingUInt64, max_columns_to_read, 0, "", 0) \
M(SettingUInt64, max_temporary_columns, 0, "", 0) \
M(SettingUInt64, max_temporary_non_const_columns, 0, "", 0) \
M(UInt64, max_columns_to_read, 0, "", 0) \
M(UInt64, max_temporary_columns, 0, "", 0) \
M(UInt64, max_temporary_non_const_columns, 0, "", 0) \
\
M(SettingUInt64, max_subquery_depth, 100, "", 0) \
M(SettingUInt64, max_pipeline_depth, 1000, "", 0) \
M(SettingUInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \
M(SettingUInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \
M(SettingUInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.", 0) \
M(UInt64, max_subquery_depth, 100, "", 0) \
M(UInt64, max_pipeline_depth, 1000, "", 0) \
M(UInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \
M(UInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \
M(UInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.", 0) \
\
M(SettingUInt64, readonly, 0, "0 - everything is allowed. 1 - only read requests. 2 - only read requests, as well as changing settings, except for the 'readonly' setting.", 0) \
M(UInt64, readonly, 0, "0 - everything is allowed. 1 - only read requests. 2 - only read requests, as well as changing settings, except for the 'readonly' setting.", 0) \
\
M(SettingUInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.", 0) \
M(SettingUInt64, max_bytes_in_set, 0, "Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.", 0) \
M(SettingOverflowMode, set_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(UInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.", 0) \
M(UInt64, max_bytes_in_set, 0, "Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.", 0) \
M(OverflowMode, set_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
\
M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \
M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \
M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \
M(SettingJoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \
M(SettingBool, partial_merge_join_optimizations, true, "Enable optimizations in partial merge join", 0) \
M(SettingUInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \
M(SettingUInt64, partial_merge_join_left_table_buffer_bytes, 32000000, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread. In current version work only with 'partial_merge_join_optimizations = 1'.", 0) \
M(SettingUInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \
M(SettingUInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \
M(SettingString, temporary_files_codec, "LZ4", "Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.", 0) \
M(UInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \
M(UInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \
M(OverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(Bool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \
M(JoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \
M(Bool, partial_merge_join_optimizations, true, "Enable optimizations in partial merge join", 0) \
M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \
M(UInt64, partial_merge_join_left_table_buffer_bytes, 32000000, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread. In current version work only with 'partial_merge_join_optimizations = 1'.", 0) \
M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \
M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \
M(String, temporary_files_codec, "LZ4", "Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.", 0) \
\
M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.", 0) \
M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.", 0) \
M(SettingOverflowMode, transfer_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(UInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.", 0) \
M(UInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.", 0) \
M(OverflowMode, transfer_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
\
M(SettingUInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.", 0) \
M(SettingUInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.", 0) \
M(SettingOverflowMode, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(UInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.", 0) \
M(UInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.", 0) \
M(OverflowMode, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
\
M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \
M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
M(SettingUInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(SettingUInt64, memory_profiler_step, 0, "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(SettingFloat, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(UInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \
M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(UInt64, memory_profiler_step, 0, "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
\
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \
M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \
M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.", 0)\
M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.", 0) \
M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \
M(UInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \
M(UInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.", 0)\
M(UInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.", 0) \
\
M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \
M(SettingBool, log_query_settings, true, "Log query settings into the query_log.", 0) \
M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \
M(SettingLogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \
M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \
M(SettingBool, allow_push_predicate_when_subquery_contains_with, 1, "Allows push predicate when subquery contains WITH clause", 0) \
\
M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.", 0) \
M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.", 0) \
M(SettingBool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \
M(Bool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \
M(Bool, log_query_settings, true, "Log query settings into the query_log.", 0) \
M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \
M(LogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
M(Bool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \
M(Bool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \
M(Bool, allow_push_predicate_when_subquery_contains_with, 1, "Allows push predicate when subquery contains WITH clause", 0) \
\
M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing", 0) \
M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \
M(SettingUInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).", 0) \
M(SettingBool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.", 0) \
M(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.", 0) \
M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.", 0) \
M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.", 0) \
M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \
M(SettingBool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \
M(SettingBool, optimize_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \
M(SettingUInt64, read_in_order_two_level_merge_threshold, 100, "Minimal number of parts to read to run preliminary merge step during multithread reading in order of primary key.", 0) \
M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \
M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \
M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \
M(UInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.", 0) \
M(Bool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.", 0) \
M(Bool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \
\
M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \
M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \
M(SettingBool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \
M(Bool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing", 0) \
M(UInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \
M(UInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).", 0) \
M(Bool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.", 0) \
M(Bool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.", 0) \
M(Bool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.", 0) \
M(Bool, enable_debug_queries, false, "Enables debug queries such as AST.", 0) \
M(Bool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \
M(Bool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \
M(Bool, optimize_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \
M(UInt64, read_in_order_two_level_merge_threshold, 100, "Minimal number of parts to read to run preliminary merge step during multithread reading in order of primary key.", 0) \
M(Bool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \
M(Bool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \
M(Bool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \
\
M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \
M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \
M(SettingBool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \
M(Bool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \
M(Bool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \
M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \
\
M(SettingSeconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \
M(SettingUInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \
M(SettingUInt64, distributed_replica_max_ignored_errors, 0, "Number of errors that will be ignored while choosing replicas", 0) \
M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \
M(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \
M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \
\
M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \
M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \
M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \
M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
M(Seconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \
M(UInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \
M(UInt64, distributed_replica_max_ignored_errors, 0, "Number of errors that will be ignored while choosing replicas", 0) \
\
M(SettingDefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \
M(SettingBool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \
M(SettingBool, show_table_uuid_in_table_create_query_if_not_nil, true, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \
M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
M(SettingBool, optimize_move_functions_out_of_any, true, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \
M(SettingBool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \
M(SettingBool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \
M(SettingBool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \
M(SettingBool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \
M(SettingBool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
M(SettingBool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \
M(SettingBool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \
M(SettingBool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
M(SettingBool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
M(Bool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \
M(Seconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \
M(UInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \
M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
\
M(SettingBool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \
M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \
M(SettingUInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \
M(SettingBool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
M(SettingUInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
M(SettingBool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \
M(SettingBool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \
M(SettingString, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \
M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \
M(Bool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \
M(Bool, show_table_uuid_in_table_create_query_if_not_nil, true, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \
M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
M(Bool, optimize_move_functions_out_of_any, true, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \
M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \
M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \
M(Bool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \
M(Bool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \
M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \
M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \
M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
\
M(SettingBool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \
M(SettingBool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \
M(SettingBool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \
M(SettingBool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \
M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \
M(Bool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \
M(UInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \
M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
M(Bool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \
M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \
M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \
\
M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \
M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \
M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \
M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\
M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \
M(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13", 0) \
M(SettingUInt64, min_count_to_compile, 0, "Obsolete setting, does nothing. Will be removed after 2020-03-13", 0) \
M(SettingBool, allow_experimental_multiple_joins_emulation, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingBool, allow_experimental_data_skipping_indices, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingBool, merge_tree_uniform_read_distribution, true, "Obsolete setting, does nothing. Will be removed after 2020-05-20", 0) \
M(SettingUInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \
M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \
M(Bool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \
M(Bool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13", 0) \
M(UInt64, min_count_to_compile, 0, "Obsolete setting, does nothing. Will be removed after 2020-03-13", 0) \
M(Bool, allow_experimental_multiple_joins_emulation, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(Bool, allow_experimental_cross_to_join_conversion, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(Bool, allow_experimental_data_skipping_indices, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(Bool, merge_tree_uniform_read_distribution, true, "Obsolete setting, does nothing. Will be removed after 2020-05-20", 0) \
M(UInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
M(Bool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \
M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \
\
M(SettingBool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \
M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0)
M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \
M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0)
#define FORMAT_FACTORY_SETTINGS(M) \
M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \
M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \
M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \
M(SettingBool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \
M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \
M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \
M(SettingBool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \
M(SettingBool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \
M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \
M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \
#define FORMAT_FACTORY_SETTINGS(M) \
M(Char, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \
M(Bool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \
M(Bool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \
M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \
M(Bool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \
M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \
M(Bool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \
M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \
M(Bool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \
\
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \
\
M(SettingBool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
M(SettingBool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
M(SettingBool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
M(SettingBool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
M(SettingBool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \
M(SettingURI, format_avro_schema_registry_url, {}, "For AvroConfluent format: Confluent Schema Registry URL.", 0) \
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \
M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \
\
M(SettingBool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \
M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \
\
M(SettingBool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \
M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \
\
M(SettingBool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \
M(Bool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \
\
M(SettingUInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \
M(SettingUInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \
M(SettingUInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \
M(SettingBool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \
M(SettingString, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \
M(SettingString, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \
M(SettingUInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \
M(SettingBool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \
M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \
M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \
M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \
M(Bool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \
M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \
M(UInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \
M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \
M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \
M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \
\
M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(UInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(Float, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
\
M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)", 0) \
M(SettingString, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)", 0) \
M(SettingString, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \
M(SettingString, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 0) \
M(String, format_schema, "", "Schema identifier (used by schema-based formats)", 0) \
M(String, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)", 0) \
M(String, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \
M(String, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 0) \
\
M(SettingString, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \
M(SettingString, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \
M(SettingString, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \
M(SettingString, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \
M(SettingString, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \
M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \
M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \
M(String, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \
M(String, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \
M(String, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \
M(String, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \
M(String, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \
M(String, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \
M(String, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \
\
M(SettingString, format_regexp, "", "Regular expression (for Regexp format)", 0) \
M(SettingString, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \
M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \
M(String, format_regexp, "", "Regular expression (for Regexp format)", 0) \
M(String, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \
M(Bool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \
\
M(SettingBool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \
M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \
M(SettingBool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \
M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \
M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \
M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \
#define LIST_OF_SETTINGS(M) \
COMMON_SETTINGS(M) \
FORMAT_FACTORY_SETTINGS(M)
#define LIST_OF_SETTINGS(M) \
COMMON_SETTINGS(M) \
FORMAT_FACTORY_SETTINGS(M)
DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS)
DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS)
/** Settings of query execution.
* These settings go to users.xml.
*/
struct Settings : public BaseSettings<SettingsTraits>
{
/// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14.
/// http://en.cppreference.com/w/cpp/language/aggregate_initialization
Settings() {}
/** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings).
* The profile can also be set using the `set` functions, like the profile setting.

View File

@ -1,612 +0,0 @@
#include <Core/SettingsCollection.h>
#include <Core/SettingsCollectionImpl.h>
#include <Core/Field.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/FieldVisitors.h>
#include <common/logger_useful.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_LOAD_BALANCING;
extern const int UNKNOWN_OVERFLOW_MODE;
extern const int UNKNOWN_TOTALS_MODE;
extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE;
extern const int UNKNOWN_JOIN;
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_SETTING;
extern const int CANNOT_PARSE_BOOL;
}
template <typename Type>
String SettingNumber<Type>::toString() const
{
return DB::toString(value);
}
template <typename Type>
Field SettingNumber<Type>::toField() const
{
return value;
}
template <typename Type>
void SettingNumber<Type>::set(Type x)
{
value = x;
changed = true;
}
template <typename Type>
void SettingNumber<Type>::set(const Field & x)
{
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(applyVisitor(FieldVisitorConvertToNumber<Type>(), x));
}
template <typename Type>
void SettingNumber<Type>::set(const String & x)
{
set(parseWithSizeSuffix<Type>(x));
}
template <>
void SettingNumber<bool>::set(const String & x)
{
if (x.size() == 1)
{
if (x[0] == '0')
set(false);
else if (x[0] == '1')
set(true);
else
throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL);
}
else
{
ReadBufferFromString buf(x);
if (checkStringCaseInsensitive("true", buf))
set(true);
else if (checkStringCaseInsensitive("false", buf))
set(false);
else
throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL);
}
}
template <typename Type>
void SettingNumber<Type>::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const
{
if (format >= SettingsBinaryFormat::STRINGS)
{
writeStringBinary(toString(), buf);
return;
}
if constexpr (is_integral_v<Type> && is_unsigned_v<Type>)
writeVarUInt(static_cast<UInt64>(value), buf);
else if constexpr (is_integral_v<Type> && is_signed_v<Type>)
writeVarInt(static_cast<Int64>(value), buf);
else
{
static_assert(std::is_floating_point_v<Type>);
writeStringBinary(toString(), buf);
}
}
template <typename Type>
void SettingNumber<Type>::deserialize(ReadBuffer & buf, SettingsBinaryFormat format)
{
if (format >= SettingsBinaryFormat::STRINGS)
{
String x;
readStringBinary(x, buf);
set(x);
return;
}
if constexpr (is_integral_v<Type> && is_unsigned_v<Type>)
{
UInt64 x;
readVarUInt(x, buf);
set(static_cast<Type>(x));
}
else if constexpr (is_integral_v<Type> && is_signed_v<Type>)
{
Int64 x;
readVarInt(x, buf);
set(static_cast<Type>(x));
}
else
{
static_assert(std::is_floating_point_v<Type>);
String x;
readStringBinary(x, buf);
set(x);
}
}
template struct SettingNumber<UInt64>;
template struct SettingNumber<Int64>;
template struct SettingNumber<float>;
template struct SettingNumber<bool>;
String SettingMaxThreads::toString() const
{
/// Instead of the `auto` value, we output the actual value to make it easier to see.
return is_auto ? ("auto(" + DB::toString(value) + ")") : DB::toString(value);
}
Field SettingMaxThreads::toField() const
{
return is_auto ? 0 : value;
}
void SettingMaxThreads::set(UInt64 x)
{
value = x ? x : getAutoValue();
is_auto = x == 0;
changed = true;
}
void SettingMaxThreads::set(const Field & x)
{
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(applyVisitor(FieldVisitorConvertToNumber<UInt64>(), x));
}
void SettingMaxThreads::set(const String & x)
{
if (startsWith(x, "auto"))
setAuto();
else
set(parse<UInt64>(x));
}
void SettingMaxThreads::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const
{
if (format >= SettingsBinaryFormat::STRINGS)
{
writeStringBinary(is_auto ? "auto" : DB::toString(value), buf);
return;
}
writeVarUInt(is_auto ? 0 : value, buf);
}
void SettingMaxThreads::deserialize(ReadBuffer & buf, SettingsBinaryFormat format)
{
if (format >= SettingsBinaryFormat::STRINGS)
{
String x;
readStringBinary(x, buf);
set(x);
return;
}
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
}
void SettingMaxThreads::setAuto()
{
value = getAutoValue();
is_auto = true;
}
UInt64 SettingMaxThreads::getAutoValue()
{
static auto res = getNumberOfPhysicalCPUCores();
return res;
}
template <SettingTimespanIO io_unit>
String SettingTimespan<io_unit>::toString() const
{
return DB::toString(value.totalMicroseconds() / microseconds_per_io_unit);
}
template <SettingTimespanIO io_unit>
Field SettingTimespan<io_unit>::toField() const
{
return value.totalMicroseconds() / microseconds_per_io_unit;
}
template <SettingTimespanIO io_unit>
void SettingTimespan<io_unit>::set(const Poco::Timespan & x)
{
value = x;
changed = true;
}
template <SettingTimespanIO io_unit>
void SettingTimespan<io_unit>::set(UInt64 x)
{
set(Poco::Timespan(x * microseconds_per_io_unit));
}
template <SettingTimespanIO io_unit>
void SettingTimespan<io_unit>::set(const Field & x)
{
if (x.getType() == Field::Types::String)
set(get<const String &>(x));
else
set(applyVisitor(FieldVisitorConvertToNumber<UInt64>(), x));
}
template <SettingTimespanIO io_unit>
void SettingTimespan<io_unit>::set(const String & x)
{
set(parse<UInt64>(x));
}
template <SettingTimespanIO io_unit>
void SettingTimespan<io_unit>::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const
{
if (format >= SettingsBinaryFormat::STRINGS)
{
writeStringBinary(toString(), buf);
return;
}
writeVarUInt(value.totalMicroseconds() / microseconds_per_io_unit, buf);
}
template <SettingTimespanIO io_unit>
void SettingTimespan<io_unit>::deserialize(ReadBuffer & buf, SettingsBinaryFormat format)
{
if (format >= SettingsBinaryFormat::STRINGS)
{
String x;
readStringBinary(x, buf);
set(x);
return;
}
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
}
template struct SettingTimespan<SettingTimespanIO::SECOND>;
template struct SettingTimespan<SettingTimespanIO::MILLISECOND>;
String SettingString::toString() const
{
return value;
}
Field SettingString::toField() const
{
return value;
}
void SettingString::set(const String & x)
{
value = x;
changed = true;
}
void SettingString::set(const Field & x)
{
set(safeGet<const String &>(x));
}
void SettingString::serialize(WriteBuffer & buf, SettingsBinaryFormat) const
{
writeStringBinary(value, buf);
}
void SettingString::deserialize(ReadBuffer & buf, SettingsBinaryFormat)
{
String s;
readStringBinary(s, buf);
set(s);
}
String SettingChar::toString() const
{
return String(1, value);
}
Field SettingChar::toField() const
{
return toString();
}
void SettingChar::set(char x)
{
value = x;
changed = true;
}
void SettingChar::set(const String & x)
{
if (x.size() > 1)
throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH);
char c = (x.size() == 1) ? x[0] : '\0';
set(c);
}
void SettingChar::set(const Field & x)
{
const String & s = safeGet<const String &>(x);
set(s);
}
void SettingChar::serialize(WriteBuffer & buf, SettingsBinaryFormat) const
{
writeStringBinary(toString(), buf);
}
void SettingChar::deserialize(ReadBuffer & buf, SettingsBinaryFormat)
{
String s;
readStringBinary(s, buf);
set(s);
}
template <typename EnumType, typename Tag>
void SettingEnum<EnumType, Tag>::serialize(WriteBuffer & buf, SettingsBinaryFormat) const
{
writeStringBinary(toString(), buf);
}
template <typename EnumType, typename Tag>
void SettingEnum<EnumType, Tag>::deserialize(ReadBuffer & buf, SettingsBinaryFormat)
{
String s;
readStringBinary(s, buf);
set(s);
}
template <typename EnumType, typename Tag>
Field SettingEnum<EnumType, Tag>::toField() const
{
return toString();
}
template <typename EnumType, typename Tag>
void SettingEnum<EnumType, Tag>::set(const Field & x)
{
set(safeGet<const String &>(x));
}
String SettingURI::toString() const
{
return value.toString();
}
Field SettingURI::toField() const
{
return value.toString();
}
void SettingURI::set(const Poco::URI & x)
{
value = x;
changed = true;
}
void SettingURI::set(const Field & x)
{
const String & s = safeGet<const String &>(x);
set(s);
}
void SettingURI::set(const String & x)
{
set(Poco::URI(x));
}
void SettingURI::serialize(WriteBuffer & buf, SettingsBinaryFormat) const
{
writeStringBinary(toString(), buf);
}
void SettingURI::deserialize(ReadBuffer & buf, SettingsBinaryFormat)
{
String s;
readStringBinary(s, buf);
set(s);
}
#define IMPLEMENT_SETTING_ENUM(ENUM_NAME, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) \
IMPLEMENT_SETTING_ENUM_WITH_TAG(ENUM_NAME, void, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME)
#define IMPLEMENT_SETTING_ENUM_WITH_TAG(ENUM_NAME, TAG, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) \
template <> \
String SettingEnum<ENUM_NAME, TAG>::toString() const \
{ \
using EnumType = ENUM_NAME; \
using UnderlyingType = std::underlying_type<EnumType>::type; \
switch (static_cast<UnderlyingType>(value)) \
{ \
LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_TO_STRING_HELPER_) \
} \
throw Exception("Unknown " #ENUM_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME); \
} \
\
template <> \
void SettingEnum<ENUM_NAME, TAG>::set(const String & s) \
{ \
using EnumType = ENUM_NAME; \
LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_FROM_STRING_HELPER_) \
\
String all_io_names; \
LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_CONCAT_NAMES_HELPER_) \
throw Exception("Unknown " #ENUM_NAME " : '" + s + "', must be one of " + all_io_names, \
ERROR_CODE_FOR_UNEXPECTED_NAME); \
} \
\
template struct SettingEnum<ENUM_NAME, TAG>;
#define IMPLEMENT_SETTING_ENUM_TO_STRING_HELPER_(NAME, IO_NAME) \
case static_cast<UnderlyingType>(EnumType::NAME): return IO_NAME;
#define IMPLEMENT_SETTING_ENUM_FROM_STRING_HELPER_(NAME, IO_NAME) \
if (s == (IO_NAME)) \
{ \
set(EnumType::NAME); \
return; \
}
#define IMPLEMENT_SETTING_ENUM_CONCAT_NAMES_HELPER_(NAME, IO_NAME) \
if (!all_io_names.empty()) \
all_io_names += ", "; \
all_io_names += String("'") + (IO_NAME) + "'";
#define LOAD_BALANCING_LIST_OF_NAMES(M) \
M(RANDOM, "random") \
M(NEAREST_HOSTNAME, "nearest_hostname") \
M(IN_ORDER, "in_order") \
M(FIRST_OR_RANDOM, "first_or_random") \
M(ROUND_ROBIN, "round_robin")
IMPLEMENT_SETTING_ENUM(LoadBalancing, LOAD_BALANCING_LIST_OF_NAMES, ErrorCodes::UNKNOWN_LOAD_BALANCING)
#define SPECIAL_SORT_ALGORITHM_NAMES(M) \
M(NOT_SPECIFIED, "not_specified") \
M(OPENCL_BITONIC, "opencl_bitonic")
IMPLEMENT_SETTING_ENUM(SpecialSort, SPECIAL_SORT_ALGORITHM_NAMES, ErrorCodes::UNKNOWN_JOIN)
#define JOIN_STRICTNESS_LIST_OF_NAMES(M) \
M(Unspecified, "") \
M(ALL, "ALL") \
M(ANY, "ANY")
IMPLEMENT_SETTING_ENUM(JoinStrictness, JOIN_STRICTNESS_LIST_OF_NAMES, ErrorCodes::UNKNOWN_JOIN) // NOLINT
#define JOIN_ALGORITHM_NAMES(M) \
M(AUTO, "auto") \
M(HASH, "hash") \
M(PARTIAL_MERGE, "partial_merge") \
M(PREFER_PARTIAL_MERGE, "prefer_partial_merge")
IMPLEMENT_SETTING_ENUM(JoinAlgorithm, JOIN_ALGORITHM_NAMES, ErrorCodes::UNKNOWN_JOIN)
#define TOTALS_MODE_LIST_OF_NAMES(M) \
M(BEFORE_HAVING, "before_having") \
M(AFTER_HAVING_EXCLUSIVE, "after_having_exclusive") \
M(AFTER_HAVING_INCLUSIVE, "after_having_inclusive") \
M(AFTER_HAVING_AUTO, "after_having_auto")
IMPLEMENT_SETTING_ENUM(TotalsMode, TOTALS_MODE_LIST_OF_NAMES, ErrorCodes::UNKNOWN_TOTALS_MODE)
#define OVERFLOW_MODE_LIST_OF_NAMES(M) \
M(THROW, "throw") \
M(BREAK, "break")
IMPLEMENT_SETTING_ENUM(OverflowMode, OVERFLOW_MODE_LIST_OF_NAMES, ErrorCodes::UNKNOWN_OVERFLOW_MODE)
#define OVERFLOW_MODE_LIST_OF_NAMES_WITH_ANY(M) \
M(THROW, "throw") \
M(BREAK, "break") \
M(ANY, "any")
IMPLEMENT_SETTING_ENUM_WITH_TAG(OverflowMode, SettingOverflowModeGroupByTag, OVERFLOW_MODE_LIST_OF_NAMES_WITH_ANY, ErrorCodes::UNKNOWN_OVERFLOW_MODE)
#define DISTRIBUTED_PRODUCT_MODE_LIST_OF_NAMES(M) \
M(DENY, "deny") \
M(LOCAL, "local") \
M(GLOBAL, "global") \
M(ALLOW, "allow")
IMPLEMENT_SETTING_ENUM(DistributedProductMode, DISTRIBUTED_PRODUCT_MODE_LIST_OF_NAMES, ErrorCodes::UNKNOWN_DISTRIBUTED_PRODUCT_MODE)
#define DATE_TIME_INPUT_FORMAT_LIST_OF_NAMES(M) \
M(Basic, "basic") \
M(BestEffort, "best_effort")
IMPLEMENT_SETTING_ENUM(FormatSettings::DateTimeInputFormat, DATE_TIME_INPUT_FORMAT_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS)
#define LOGS_LEVEL_LIST_OF_NAMES(M) \
M(none, "none") \
M(fatal, "fatal") \
M(error, "error") \
M(warning, "warning") \
M(information, "information") \
M(debug, "debug") \
M(trace, "trace")
IMPLEMENT_SETTING_ENUM(LogsLevel, LOGS_LEVEL_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS)
#define LOG_QUERIES_TYPE_LIST_OF_NAMES(M) \
M(QUERY_START, "QUERY_START") \
M(QUERY_FINISH, "QUERY_FINISH") \
M(EXCEPTION_BEFORE_START, "EXCEPTION_BEFORE_START") \
M(EXCEPTION_WHILE_PROCESSING, "EXCEPTION_WHILE_PROCESSING")
IMPLEMENT_SETTING_ENUM(QueryLogElementType, LOG_QUERIES_TYPE_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS)
#define DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES(M) \
M(Ordinary, "Ordinary") \
M(Atomic, "Atomic")
IMPLEMENT_SETTING_ENUM(DefaultDatabaseEngine , DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS)
namespace details
{
void SettingsCollectionUtils::serializeName(const StringRef & name, WriteBuffer & buf)
{
writeStringBinary(name, buf);
}
String SettingsCollectionUtils::deserializeName(ReadBuffer & buf)
{
String name;
readStringBinary(name, buf);
return name;
}
void SettingsCollectionUtils::serializeFlag(bool flag, WriteBuffer & buf)
{
buf.write(flag);
}
bool SettingsCollectionUtils::deserializeFlag(ReadBuffer & buf)
{
char c;
buf.readStrict(c);
return c;
}
void SettingsCollectionUtils::skipValue(ReadBuffer & buf)
{
/// Ignore a string written by the function writeStringBinary().
UInt64 size;
readVarUInt(size, buf);
buf.ignore(size);
}
void SettingsCollectionUtils::warningNameNotFound(const StringRef & name)
{
static auto * log = &Poco::Logger::get("Settings");
LOG_WARNING(log, "Unknown setting {}, skipping", name);
}
void SettingsCollectionUtils::throwNameNotFound(const StringRef & name)
{
throw Exception("Unknown setting " + name.toString(), ErrorCodes::UNKNOWN_SETTING);
}
}
}

View File

@ -1,581 +0,0 @@
#pragma once
#include <Poco/Timespan.h>
#include <Poco/URI.h>
#include <DataStreams/SizeLimits.h>
#include <Formats/FormatSettings.h>
#include <common/StringRef.h>
#include <Core/Types.h>
#include <unordered_map>
namespace DB
{
class Field;
struct SettingChange;
using SettingsChanges = std::vector<SettingChange>;
class ReadBuffer;
class WriteBuffer;
enum class SettingsBinaryFormat;
/** One setting for any type.
* Stores a value within itself, as well as a flag - whether the value was changed.
* This is done so that you can send to the remote servers only changed settings (or explicitly specified in the config) values.
* That is, if the configuration was not specified in the config and was not dynamically changed, it is not sent to the remote server,
* and the remote server will use its default value.
*/
template <typename Type>
struct SettingNumber
{
Type value;
bool changed = false;
SettingNumber(Type x = 0) : value(x) {}
operator Type() const { return value; }
SettingNumber & operator= (Type x) { set(x); return *this; }
/// Serialize to a test string.
String toString() const;
/// Converts to a field.
Field toField() const;
void set(Type x);
/// Read from SQL literal.
void set(const Field & x);
/// Read from text string.
void set(const String & x);
/// Serialize to binary stream suitable for transfer over network.
void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const;
/// Read from binary stream.
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
};
using SettingUInt64 = SettingNumber<UInt64>;
using SettingInt64 = SettingNumber<Int64>;
using SettingFloat = SettingNumber<float>;
using SettingBool = SettingNumber<bool>;
/** Unlike SettingUInt64, supports the value of 'auto' - the number of processor cores without taking into account SMT.
* A value of 0 is also treated as auto.
* When serializing, `auto` is written in the same way as 0.
*/
struct SettingMaxThreads
{
UInt64 value;
bool is_auto;
bool changed = false;
SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {}
operator UInt64() const { return value; }
SettingMaxThreads & operator= (UInt64 x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(UInt64 x);
void set(const Field & x);
void set(const String & x);
void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const;
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
void setAuto();
static UInt64 getAutoValue();
};
enum class SettingTimespanIO { MILLISECOND, SECOND };
template <SettingTimespanIO io_unit>
struct SettingTimespan
{
Poco::Timespan value;
bool changed = false;
SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {}
operator Poco::Timespan() const { return value; }
SettingTimespan & operator= (const Poco::Timespan & x) { set(x); return *this; }
Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); }
Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); }
String toString() const;
Field toField() const;
void set(const Poco::Timespan & x);
void set(UInt64 x);
void set(const Field & x);
void set(const String & x);
void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const;
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
static constexpr UInt64 microseconds_per_io_unit = (io_unit == SettingTimespanIO::MILLISECOND) ? 1000 : 1000000;
};
using SettingSeconds = SettingTimespan<SettingTimespanIO::SECOND>;
using SettingMilliseconds = SettingTimespan<SettingTimespanIO::MILLISECOND>;
struct SettingString
{
String value;
bool changed = false;
SettingString(const String & x = String{}) : value(x) {}
operator String() const { return value; }
SettingString & operator= (const String & x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(const String & x);
void set(const Field & x);
void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const;
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
};
struct SettingChar
{
public:
char value;
bool changed = false;
SettingChar(char x = '\0') : value(x) {}
operator char() const { return value; }
SettingChar & operator= (char x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(char x);
void set(const String & x);
void set(const Field & x);
void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const;
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
};
/// Template class to define enum-based settings.
template <typename EnumType, typename Tag = void>
struct SettingEnum
{
EnumType value;
bool changed = false;
SettingEnum(EnumType x) : value(x) {}
operator EnumType() const { return value; }
SettingEnum & operator= (EnumType x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(EnumType x) { value = x; changed = true; }
void set(const Field & x);
void set(const String & x);
void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const;
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
};
struct SettingURI
{
Poco::URI value;
bool changed = false;
SettingURI(const Poco::URI & x = Poco::URI{}) : value(x) {}
operator Poco::URI() const { return value; }
SettingURI & operator= (const Poco::URI & x) { set(x); return *this; }
String toString() const;
Field toField() const;
void set(const Poco::URI & x);
void set(const Field & x);
void set(const String & x);
void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const;
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
};
enum class LoadBalancing
{
/// among replicas with a minimum number of errors selected randomly
RANDOM = 0,
/// a replica is selected among the replicas with the minimum number of errors
/// with the minimum number of distinguished characters in the replica name and local hostname
NEAREST_HOSTNAME,
// replicas with the same number of errors are accessed in the same order
// as they are specified in the configuration.
IN_ORDER,
/// if first replica one has higher number of errors,
/// pick a random one from replicas with minimum number of errors
FIRST_OR_RANDOM,
// round robin across replicas with the same number of errors.
ROUND_ROBIN,
};
using SettingLoadBalancing = SettingEnum<LoadBalancing>;
enum class JoinStrictness
{
Unspecified = 0, /// Query JOIN without strictness will throw Exception.
ALL, /// Query JOIN without strictness -> ALL JOIN ...
ANY, /// Query JOIN without strictness -> ANY JOIN ...
};
using SettingJoinStrictness = SettingEnum<JoinStrictness>;
enum class JoinAlgorithm
{
AUTO = 0,
HASH,
PARTIAL_MERGE,
PREFER_PARTIAL_MERGE,
};
using SettingJoinAlgorithm = SettingEnum<JoinAlgorithm>;
enum class SpecialSort
{
NOT_SPECIFIED = 0,
OPENCL_BITONIC,
};
using SettingSpecialSort = SettingEnum<SpecialSort>;
/// Which rows should be included in TOTALS.
enum class TotalsMode
{
BEFORE_HAVING = 0, /// Count HAVING for all read rows;
/// including those not in max_rows_to_group_by
/// and have not passed HAVING after grouping.
AFTER_HAVING_INCLUSIVE = 1, /// Count on all rows except those that have not passed HAVING;
/// that is, to include in TOTALS all the rows that did not pass max_rows_to_group_by.
AFTER_HAVING_EXCLUSIVE = 2, /// Include only the rows that passed and max_rows_to_group_by, and HAVING.
AFTER_HAVING_AUTO = 3, /// Automatically select between INCLUSIVE and EXCLUSIVE,
};
using SettingTotalsMode = SettingEnum<TotalsMode>;
/// The settings keeps OverflowMode which cannot be OverflowMode::ANY.
using SettingOverflowMode = SettingEnum<OverflowMode>;
struct SettingOverflowModeGroupByTag;
/// The settings keeps OverflowMode which can be OverflowMode::ANY.
using SettingOverflowModeGroupBy = SettingEnum<OverflowMode, SettingOverflowModeGroupByTag>;
/// The setting for executing distributed subqueries inside IN or JOIN sections.
enum class DistributedProductMode
{
DENY = 0, /// Disable
LOCAL, /// Convert to local query
GLOBAL, /// Convert to global query
ALLOW /// Enable
};
using SettingDistributedProductMode = SettingEnum<DistributedProductMode>;
using SettingDateTimeInputFormat = SettingEnum<FormatSettings::DateTimeInputFormat>;
enum class LogsLevel
{
none = 0, /// Disable
fatal,
error,
warning,
information,
debug,
trace,
};
using SettingLogsLevel = SettingEnum<LogsLevel>;
enum class DefaultDatabaseEngine
{
Ordinary,
Atomic,
};
using SettingDefaultDatabaseEngine = SettingEnum<DefaultDatabaseEngine>;
// Make it signed for compatibility with DataTypeEnum8
enum QueryLogElementType : int8_t
{
QUERY_START = 1,
QUERY_FINISH = 2,
EXCEPTION_BEFORE_START = 3,
EXCEPTION_WHILE_PROCESSING = 4,
};
using SettingLogQueriesType = SettingEnum<QueryLogElementType>;
enum class SettingsBinaryFormat
{
OLD, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour.
STRINGS, /// All settings are serialized as strings. Before each value the flag `is_ignorable` is serialized.
DEFAULT = STRINGS,
};
/** Template class to define collections of settings.
* Example of usage:
*
* mysettings.h:
* struct MySettings : public SettingsCollection<MySettings>
* {
* # define APPLY_FOR_MYSETTINGS(M) \
* M(SettingUInt64, a, 100, "Description of a", 0) \
* M(SettingFloat, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \
* M(SettingString, s, "default", "Description of s", 0)
*
* DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS)
* };
*
* mysettings.cpp:
* IMPLEMENT_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS)
*/
template <class Derived>
class SettingsCollection
{
private:
Derived & castToDerived() { return *static_cast<Derived *>(this); }
const Derived & castToDerived() const { return *static_cast<const Derived *>(this); }
struct MemberInfo
{
using IsChangedFunction = bool (*)(const Derived &);
using GetStringFunction = String (*)(const Derived &);
using GetFieldFunction = Field (*)(const Derived &);
using SetStringFunction = void (*)(Derived &, const String &);
using SetFieldFunction = void (*)(Derived &, const Field &);
using SerializeFunction = void (*)(const Derived &, WriteBuffer & buf, SettingsBinaryFormat);
using DeserializeFunction = void (*)(Derived &, ReadBuffer & buf, SettingsBinaryFormat);
using ValueToStringFunction = String (*)(const Field &);
using ValueToCorrespondingTypeFunction = Field (*)(const Field &);
StringRef name;
StringRef description;
StringRef type;
bool is_important;
IsChangedFunction is_changed;
GetStringFunction get_string;
GetFieldFunction get_field;
SetStringFunction set_string;
SetFieldFunction set_field;
SerializeFunction serialize;
DeserializeFunction deserialize;
ValueToStringFunction value_to_string;
ValueToCorrespondingTypeFunction value_to_corresponding_type;
};
class MemberInfos
{
public:
MemberInfos();
size_t size() const { return infos.size(); }
const MemberInfo * data() const { return infos.data(); }
const MemberInfo & operator[](size_t index) const { return infos[index]; }
const MemberInfo * find(const StringRef & name) const;
const MemberInfo & findStrict(const StringRef & name) const;
size_t findIndex(const StringRef & name) const;
size_t findIndexStrict(const StringRef & name) const;
private:
void add(MemberInfo && member);
std::vector<MemberInfo> infos;
std::unordered_map<StringRef, size_t> by_name_map;
};
static const MemberInfos & members();
public:
class const_iterator;
/// Provides read-only access to a setting.
class const_reference
{
public:
const_reference(const Derived & collection_, const MemberInfo & member_) : collection(&collection_), member(&member_) {}
const_reference(const const_reference & src) = default;
const StringRef & getName() const { return member->name; }
const StringRef & getDescription() const { return member->description; }
const StringRef & getType() const { return member->type; }
bool isChanged() const { return member->is_changed(*collection); }
Field getValue() const;
String getValueAsString() const { return member->get_string(*collection); }
protected:
friend class SettingsCollection<Derived>::const_iterator;
const_reference() : collection(nullptr), member(nullptr) {}
const_reference & operator=(const const_reference &) = default;
const Derived * collection;
const MemberInfo * member;
};
/// Provides access to a setting.
class reference : public const_reference
{
public:
reference(Derived & collection_, const MemberInfo & member_) : const_reference(collection_, member_) {}
reference(const const_reference & src) : const_reference(src) {}
void setValue(const Field & value) { this->member->set_field(*const_cast<Derived *>(this->collection), value); }
void setValue(const String & value) { this->member->set_string(*const_cast<Derived *>(this->collection), value); }
};
/// Iterator to iterating through all the settings.
class const_iterator
{
public:
const_iterator(const Derived & collection_, const MemberInfo * member_) : ref(const_cast<Derived &>(collection_), *member_) {}
const_iterator() = default;
const_iterator(const const_iterator & src) = default;
const_iterator & operator =(const const_iterator & src) = default;
const const_reference & operator *() const { return ref; }
const const_reference * operator ->() const { return &ref; }
const_iterator & operator ++() { ++ref.member; return *this; }
const_iterator operator ++(int) { const_iterator tmp = *this; ++*this; return tmp; }
bool operator ==(const const_iterator & rhs) const { return ref.member == rhs.ref.member && ref.collection == rhs.ref.collection; }
bool operator !=(const const_iterator & rhs) const { return !(*this == rhs); }
protected:
mutable reference ref;
};
class iterator : public const_iterator
{
public:
iterator(Derived & collection_, const MemberInfo * member_) : const_iterator(collection_, member_) {}
iterator() = default;
iterator(const const_iterator & src) : const_iterator(src) {}
iterator & operator =(const const_iterator & src) { const_iterator::operator =(src); return *this; }
reference & operator *() const { return this->ref; }
reference * operator ->() const { return &this->ref; }
iterator & operator ++() { const_iterator::operator ++(); return *this; }
iterator operator ++(int) { iterator tmp = *this; ++*this; return tmp; }
};
/// Returns the number of settings.
static size_t size() { return members().size(); }
/// Returns name of a setting by its index (0..size()-1).
static StringRef getName(size_t index) { return members()[index].name; }
/// Returns description of a setting.
static StringRef getDescription(size_t index) { return members()[index].description; }
static StringRef getDescription(const String & name) { return members().findStrict(name).description; }
/// Searches a setting by its name; returns `npos` if not found.
static size_t findIndex(const StringRef & name) { return members().findIndex(name); }
static constexpr size_t npos = static_cast<size_t>(-1);
/// Searches a setting by its name; throws an exception if not found.
static size_t findIndexStrict(const StringRef & name) { return members().findIndexStrict(name); }
/// Casts a value to a string according to a specified setting without actual changing this settings.
static String valueToString(size_t index, const Field & value) { return members()[index].value_to_string(value); }
static String valueToString(const StringRef & name, const Field & value) { return members().findStrict(name).value_to_string(value); }
/// Casts a value to a type according to a specified setting without actual changing this settings.
/// E.g. for SettingInt64 it casts Field to Field::Types::Int64.
static Field valueToCorrespondingType(size_t index, const Field & value);
static Field valueToCorrespondingType(const StringRef & name, const Field & value);
iterator begin() { return iterator(castToDerived(), members().data()); }
const_iterator begin() const { return const_iterator(castToDerived(), members().data()); }
iterator end() { const auto & the_members = members(); return iterator(castToDerived(), the_members.data() + the_members.size()); }
const_iterator end() const { const auto & the_members = members(); return const_iterator(castToDerived(), the_members.data() + the_members.size()); }
/// Returns a proxy object for accessing to a setting. Throws an exception if there is not setting with such name.
reference operator[](size_t index) { return reference(castToDerived(), members()[index]); }
reference operator[](const StringRef & name) { return reference(castToDerived(), members().findStrict(name)); }
const_reference operator[](size_t index) const { return const_reference(castToDerived(), members()[index]); }
const_reference operator[](const StringRef & name) const { return const_reference(castToDerived(), members().findStrict(name)); }
/// Searches a setting by its name; returns end() if not found.
iterator find(const StringRef & name);
const_iterator find(const StringRef & name) const;
/// Searches a setting by its name; throws an exception if not found.
iterator findStrict(const StringRef & name);
const_iterator findStrict(const StringRef & name) const;
/// Sets setting's value.
void set(size_t index, const Field & value) { (*this)[index].setValue(value); }
void set(const StringRef & name, const Field & value) { (*this)[name].setValue(value); }
/// Sets setting's value. Read value in text form from string (for example, from configuration file or from URL parameter).
void set(size_t index, const String & value) { (*this)[index].setValue(value); }
void set(const StringRef & name, const String & value) { (*this)[name].setValue(value); }
/// Returns value of a setting.
Field get(size_t index) const;
Field get(const StringRef & name) const;
/// Returns value of a setting converted to string.
String getAsString(size_t index) const { return (*this)[index].getValueAsString(); }
String getAsString(const StringRef & name) const { return (*this)[name].getValueAsString(); }
/// Returns value of a setting; returns false if there is no setting with the specified name.
bool tryGet(const StringRef & name, Field & value) const;
/// Returns value of a setting converted to string; returns false if there is no setting with the specified name.
bool tryGet(const StringRef & name, String & value) const;
/// Compares two collections of settings.
bool operator ==(const SettingsCollection & rhs) const;
bool operator!=(const SettingsCollection & rhs) const { return !(*this == rhs); }
/// Gathers all changed values (e.g. for applying them later to another collection of settings).
SettingsChanges changes() const;
// A debugging aid.
std::string dumpChangesToString() const;
/// Applies change to concrete setting.
void applyChange(const SettingChange & change);
/// Applies changes to the settings.
void applyChanges(const SettingsChanges & changes);
void copyChangesFrom(const Derived & src);
void copyChangesTo(Derived & dest) const;
/// Writes the settings to buffer (e.g. to be sent to remote server).
/// Only changed settings are written. They are written as list of contiguous name-value pairs,
/// finished with empty name.
void serialize(WriteBuffer & buf, SettingsBinaryFormat format = SettingsBinaryFormat::DEFAULT) const;
/// Reads the settings from buffer.
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format = SettingsBinaryFormat::DEFAULT);
};
#define DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS_MACRO) \
LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_)
#define DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
TYPE NAME {DEFAULT};
}

View File

@ -1,359 +0,0 @@
#pragma once
/**
* This file implements some functions that are dependent on Field type.
* Unlike SettingsCollection.h, we only have to include it once for each
* instantiation of SettingsCollection<>.
*/
#include <Common/SettingsChanges.h>
#include <Common/FieldVisitors.h>
namespace DB
{
namespace details
{
struct SettingsCollectionUtils
{
static void serializeName(const StringRef & name, WriteBuffer & buf);
static String deserializeName(ReadBuffer & buf);
static void serializeFlag(bool flag, WriteBuffer & buf);
static bool deserializeFlag(ReadBuffer & buf);
static void skipValue(ReadBuffer & buf);
static void warningNameNotFound(const StringRef & name);
[[noreturn]] static void throwNameNotFound(const StringRef & name);
};
}
template <class Derived>
size_t SettingsCollection<Derived>::MemberInfos::findIndex(const StringRef & name) const
{
auto it = by_name_map.find(name);
if (it == by_name_map.end())
return static_cast<size_t>(-1); // npos
return it->second;
}
template <class Derived>
size_t SettingsCollection<Derived>::MemberInfos::findIndexStrict(const StringRef & name) const
{
auto it = by_name_map.find(name);
if (it == by_name_map.end())
details::SettingsCollectionUtils::throwNameNotFound(name);
return it->second;
}
template <class Derived>
const typename SettingsCollection<Derived>::MemberInfo * SettingsCollection<Derived>::MemberInfos::find(const StringRef & name) const
{
auto it = by_name_map.find(name);
if (it == by_name_map.end())
return nullptr;
else
return &infos[it->second];
}
template <class Derived>
const typename SettingsCollection<Derived>::MemberInfo & SettingsCollection<Derived>::MemberInfos::findStrict(const StringRef & name) const
{
return infos[findIndexStrict(name)];
}
template <class Derived>
void SettingsCollection<Derived>::MemberInfos::add(MemberInfo && member)
{
size_t index = infos.size();
infos.emplace_back(member);
by_name_map.emplace(infos.back().name, index);
}
template <class Derived>
const typename SettingsCollection<Derived>::MemberInfos &
SettingsCollection<Derived>::members()
{
static const MemberInfos the_instance;
return the_instance;
}
template <class Derived>
Field SettingsCollection<Derived>::const_reference::getValue() const
{
return member->get_field(*collection);
}
template <class Derived>
Field SettingsCollection<Derived>::valueToCorrespondingType(size_t index, const Field & value)
{
try
{
return members()[index].value_to_corresponding_type(value);
}
catch (Exception & e)
{
e.addMessage(fmt::format("in attempt to set the value of setting to {}",
applyVisitor(FieldVisitorToString(), value)));
throw;
}
}
template <class Derived>
Field SettingsCollection<Derived>::valueToCorrespondingType(const StringRef & name, const Field & value)
{
return members().findStrict(name).value_to_corresponding_type(value);
}
template <class Derived>
typename SettingsCollection<Derived>::iterator SettingsCollection<Derived>::find(const StringRef & name)
{
const auto * member = members().find(name);
if (member)
return iterator(castToDerived(), member);
return end();
}
template <class Derived>
typename SettingsCollection<Derived>::const_iterator SettingsCollection<Derived>::find(const StringRef & name) const
{
const auto * member = members().find(name);
if (member)
return const_iterator(castToDerived(), member);
return end();
}
template <class Derived>
typename SettingsCollection<Derived>::iterator SettingsCollection<Derived>::findStrict(const StringRef & name)
{
return iterator(castToDerived(), &members().findStrict(name));
}
template <class Derived>
typename SettingsCollection<Derived>::const_iterator SettingsCollection<Derived>::findStrict(const StringRef & name) const
{
return const_iterator(castToDerived(), &members().findStrict(name));
}
template <class Derived>
Field SettingsCollection<Derived>::get(size_t index) const
{
return (*this)[index].getValue();
}
template <class Derived>
Field SettingsCollection<Derived>::get(const StringRef & name) const
{
return (*this)[name].getValue();
}
template <class Derived>
bool SettingsCollection<Derived>::tryGet(const StringRef & name, Field & value) const
{
auto it = find(name);
if (it == end())
return false;
value = it->getValue();
return true;
}
template <class Derived>
bool SettingsCollection<Derived>::tryGet(const StringRef & name, String & value) const
{
auto it = find(name);
if (it == end())
return false;
value = it->getValueAsString();
return true;
}
template <class Derived>
bool SettingsCollection<Derived>::operator ==(const SettingsCollection<Derived> & rhs) const
{
const auto & the_members = members();
for (size_t i = 0; i != the_members.size(); ++i)
{
const auto & member = the_members[i];
bool left_changed = member.is_changed(castToDerived());
bool right_changed = member.is_changed(rhs.castToDerived());
if (left_changed || right_changed)
{
if (left_changed != right_changed)
return false;
if (member.get_field(castToDerived()) != member.get_field(rhs.castToDerived()))
return false;
}
}
return true;
}
template <class Derived>
SettingsChanges SettingsCollection<Derived>::changes() const
{
SettingsChanges found_changes;
const auto & the_members = members();
for (size_t i = 0; i != the_members.size(); ++i)
{
const auto & member = the_members[i];
if (member.is_changed(castToDerived()))
found_changes.push_back({member.name.toString(), member.get_field(castToDerived())});
}
return found_changes;
}
template <class Derived>
std::string SettingsCollection<Derived>::dumpChangesToString() const
{
std::stringstream ss;
for (const auto & c : changes())
{
ss << c.name << " = "
<< applyVisitor(FieldVisitorToString(), c.value) << "\n";
}
return ss.str();
}
template <class Derived>
void SettingsCollection<Derived>::applyChange(const SettingChange & change)
{
set(change.name, change.value);
}
template <class Derived>
void SettingsCollection<Derived>::applyChanges(const SettingsChanges & changes)
{
for (const SettingChange & change : changes)
applyChange(change);
}
template <class Derived>
void SettingsCollection<Derived>::copyChangesFrom(const Derived & src)
{
const auto & the_members = members();
for (size_t i = 0; i != the_members.size(); ++i)
{
const auto & member = the_members[i];
if (member.is_changed(src))
member.set_field(castToDerived(), member.get_field(src));
}
}
template <class Derived>
void SettingsCollection<Derived>::copyChangesTo(Derived & dest) const
{
dest.copyChangesFrom(castToDerived());
}
template <class Derived>
void SettingsCollection<Derived>::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const
{
const auto & the_members = members();
for (size_t i = 0; i != the_members.size(); ++i)
{
const auto & member = the_members[i];
if (member.is_changed(castToDerived()))
{
details::SettingsCollectionUtils::serializeName(member.name, buf);
if (format >= SettingsBinaryFormat::STRINGS)
details::SettingsCollectionUtils::serializeFlag(member.is_important, buf);
member.serialize(castToDerived(), buf, format);
}
}
details::SettingsCollectionUtils::serializeName(StringRef{} /* empty string is a marker of the end of settings */, buf);
}
template <class Derived>
void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFormat format)
{
const auto & the_members = members();
while (true)
{
String name = details::SettingsCollectionUtils::deserializeName(buf);
if (name.empty() /* empty string is a marker of the end of settings */)
break;
auto * member = the_members.find(name);
bool is_important = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : true;
if (member)
{
member->deserialize(castToDerived(), buf, format);
}
else if (is_important)
{
details::SettingsCollectionUtils::throwNameNotFound(name);
}
else
{
details::SettingsCollectionUtils::warningNameNotFound(name);
details::SettingsCollectionUtils::skipValue(buf);
}
}
}
//-V:IMPLEMENT_SETTINGS_COLLECTION:501
#define IMPLEMENT_SETTINGS_COLLECTION(DERIVED_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \
template<> \
SettingsCollection<DERIVED_CLASS_NAME>::MemberInfos::MemberInfos() \
{ \
using Derived = DERIVED_CLASS_NAME; \
struct Functions \
{ \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \
}; \
constexpr int IMPORTANT = 1; \
UNUSED(IMPORTANT); \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \
} \
/** \
* Instantiation should happen when all method definitions from SettingsCollectionImpl.h \
* are accessible, so we instantiate explicitly. \
*/ \
template class SettingsCollection<DERIVED_CLASS_NAME>;
#define IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
static String NAME##_getString(const Derived & collection) { return collection.NAME.toString(); } \
static Field NAME##_getField(const Derived & collection) { return collection.NAME.toField(); } \
static void NAME##_setString(Derived & collection, const String & value) { collection.NAME.set(value); } \
static void NAME##_setField(Derived & collection, const Field & value) { collection.NAME.set(value); } \
static void NAME##_serialize(const Derived & collection, WriteBuffer & buf, SettingsBinaryFormat format) { collection.NAME.serialize(buf, format); } \
static void NAME##_deserialize(Derived & collection, ReadBuffer & buf, SettingsBinaryFormat format) { collection.NAME.deserialize(buf, format); } \
static String NAME##_valueToString(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \
static Field NAME##_valueToCorrespondingType(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
add({StringRef(#NAME, strlen(#NAME)), \
StringRef(DESCRIPTION, strlen(DESCRIPTION)), \
StringRef(#TYPE, strlen(#TYPE)), \
FLAGS & IMPORTANT, \
[](const Derived & d) { return d.NAME.changed; }, \
&Functions::NAME##_getString, &Functions::NAME##_getField, \
&Functions::NAME##_setString, &Functions::NAME##_setField, \
&Functions::NAME##_serialize, &Functions::NAME##_deserialize, \
&Functions::NAME##_valueToString, &Functions::NAME##_valueToCorrespondingType});
}

View File

@ -0,0 +1,94 @@
#include <Core/SettingsEnums.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_LOAD_BALANCING;
extern const int UNKNOWN_OVERFLOW_MODE;
extern const int UNKNOWN_TOTALS_MODE;
extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE;
extern const int UNKNOWN_JOIN;
extern const int BAD_ARGUMENTS;
}
IMPLEMENT_SETTING_ENUM(LoadBalancing, ErrorCodes::UNKNOWN_LOAD_BALANCING,
{{"random", LoadBalancing::RANDOM},
{"nearest_hostname", LoadBalancing::NEAREST_HOSTNAME},
{"in_order", LoadBalancing::IN_ORDER},
{"first_or_random", LoadBalancing::FIRST_OR_RANDOM},
{"round_robin", LoadBalancing::ROUND_ROBIN}})
IMPLEMENT_SETTING_ENUM(SpecialSort, ErrorCodes::UNKNOWN_JOIN,
{{"not_specified", SpecialSort::NOT_SPECIFIED},
{"opencl_bitonic", SpecialSort::OPENCL_BITONIC}})
IMPLEMENT_SETTING_ENUM(JoinStrictness, ErrorCodes::UNKNOWN_JOIN,
{{"", JoinStrictness::Unspecified},
{"ALL", JoinStrictness::ALL},
{"ANY", JoinStrictness::ANY}})
IMPLEMENT_SETTING_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN,
{{"auto", JoinAlgorithm::AUTO},
{"hash", JoinAlgorithm::HASH},
{"partial_merge", JoinAlgorithm::PARTIAL_MERGE},
{"prefer_partial_merge", JoinAlgorithm::PREFER_PARTIAL_MERGE}})
IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE,
{{"before_having", TotalsMode::BEFORE_HAVING},
{"after_having_exclusive", TotalsMode::AFTER_HAVING_EXCLUSIVE},
{"after_having_inclusive", TotalsMode::AFTER_HAVING_INCLUSIVE},
{"after_having_auto", TotalsMode::AFTER_HAVING_AUTO}})
IMPLEMENT_SETTING_ENUM(OverflowMode, ErrorCodes::UNKNOWN_OVERFLOW_MODE,
{{"throw", OverflowMode::THROW},
{"break", OverflowMode::BREAK}})
IMPLEMENT_SETTING_ENUM_WITH_RENAME(OverflowModeGroupBy, ErrorCodes::UNKNOWN_OVERFLOW_MODE,
{{"throw", OverflowMode::THROW},
{"break", OverflowMode::BREAK},
{"any", OverflowMode::ANY}})
IMPLEMENT_SETTING_ENUM(DistributedProductMode, ErrorCodes::UNKNOWN_DISTRIBUTED_PRODUCT_MODE,
{{"deny", DistributedProductMode::DENY},
{"local", DistributedProductMode::LOCAL},
{"global", DistributedProductMode::GLOBAL},
{"allow", DistributedProductMode::ALLOW}})
IMPLEMENT_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, ErrorCodes::BAD_ARGUMENTS,
{{"basic", FormatSettings::DateTimeInputFormat::Basic},
{"best_effort", FormatSettings::DateTimeInputFormat::BestEffort}})
IMPLEMENT_SETTING_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS,
{{"none", LogsLevel::none},
{"fatal", LogsLevel::fatal},
{"error", LogsLevel::error},
{"warning", LogsLevel::warning},
{"information", LogsLevel::information},
{"debug", LogsLevel::debug},
{"trace", LogsLevel::trace}})
IMPLEMENT_SETTING_ENUM_WITH_RENAME(LogQueriesType, ErrorCodes::BAD_ARGUMENTS,
{{"QUERY_START", QUERY_START},
{"QUERY_FINISH", QUERY_FINISH},
{"EXCEPTION_BEFORE_START", EXCEPTION_BEFORE_START},
{"EXCEPTION_WHILE_PROCESSING", EXCEPTION_WHILE_PROCESSING}})
IMPLEMENT_SETTING_ENUM_WITH_RENAME(DefaultDatabaseEngine, ErrorCodes::BAD_ARGUMENTS,
{{"Ordinary", DefaultDatabaseEngine::Ordinary},
{"Atomic", DefaultDatabaseEngine::Atomic}})
}

129
src/Core/SettingsEnums.h Normal file
View File

@ -0,0 +1,129 @@
#pragma once
#include <Core/SettingsFields.h>
#include <DataStreams/SizeLimits.h>
#include <Formats/FormatSettings.h>
namespace DB
{
enum class LoadBalancing
{
/// among replicas with a minimum number of errors selected randomly
RANDOM = 0,
/// a replica is selected among the replicas with the minimum number of errors
/// with the minimum number of distinguished characters in the replica name and local hostname
NEAREST_HOSTNAME,
// replicas with the same number of errors are accessed in the same order
// as they are specified in the configuration.
IN_ORDER,
/// if first replica one has higher number of errors,
/// pick a random one from replicas with minimum number of errors
FIRST_OR_RANDOM,
// round robin across replicas with the same number of errors.
ROUND_ROBIN,
};
DECLARE_SETTING_ENUM(LoadBalancing)
enum class JoinStrictness
{
Unspecified = 0, /// Query JOIN without strictness will throw Exception.
ALL, /// Query JOIN without strictness -> ALL JOIN ...
ANY, /// Query JOIN without strictness -> ANY JOIN ...
};
DECLARE_SETTING_ENUM(JoinStrictness)
enum class JoinAlgorithm
{
AUTO = 0,
HASH,
PARTIAL_MERGE,
PREFER_PARTIAL_MERGE,
};
DECLARE_SETTING_ENUM(JoinAlgorithm)
enum class SpecialSort
{
NOT_SPECIFIED = 0,
OPENCL_BITONIC,
};
DECLARE_SETTING_ENUM(SpecialSort)
/// Which rows should be included in TOTALS.
enum class TotalsMode
{
BEFORE_HAVING = 0, /// Count HAVING for all read rows;
/// including those not in max_rows_to_group_by
/// and have not passed HAVING after grouping.
AFTER_HAVING_INCLUSIVE = 1, /// Count on all rows except those that have not passed HAVING;
/// that is, to include in TOTALS all the rows that did not pass max_rows_to_group_by.
AFTER_HAVING_EXCLUSIVE = 2, /// Include only the rows that passed and max_rows_to_group_by, and HAVING.
AFTER_HAVING_AUTO = 3, /// Automatically select between INCLUSIVE and EXCLUSIVE,
};
DECLARE_SETTING_ENUM(TotalsMode)
/// The settings keeps OverflowMode which cannot be OverflowMode::ANY.
DECLARE_SETTING_ENUM(OverflowMode)
/// The settings keeps OverflowMode which can be OverflowMode::ANY.
DECLARE_SETTING_ENUM_WITH_RENAME(OverflowModeGroupBy, OverflowMode)
/// The setting for executing distributed subqueries inside IN or JOIN sections.
enum class DistributedProductMode
{
DENY = 0, /// Disable
LOCAL, /// Convert to local query
GLOBAL, /// Convert to global query
ALLOW /// Enable
};
DECLARE_SETTING_ENUM(DistributedProductMode)
DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeInputFormat)
enum class LogsLevel
{
none = 0, /// Disable
fatal,
error,
warning,
information,
debug,
trace,
};
DECLARE_SETTING_ENUM(LogsLevel)
// Make it signed for compatibility with DataTypeEnum8
enum QueryLogElementType : int8_t
{
QUERY_START = 1,
QUERY_FINISH = 2,
EXCEPTION_BEFORE_START = 3,
EXCEPTION_WHILE_PROCESSING = 4,
};
DECLARE_SETTING_ENUM_WITH_RENAME(LogQueriesType, QueryLogElementType)
enum class DefaultDatabaseEngine
{
Ordinary,
Atomic,
};
DECLARE_SETTING_ENUM(DefaultDatabaseEngine)
}

329
src/Core/SettingsFields.cpp Normal file
View File

@ -0,0 +1,329 @@
#include <Core/SettingsFields.h>
#include <Core/Field.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/FieldVisitors.h>
#include <common/logger_useful.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <boost/algorithm/string/predicate.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
extern const int CANNOT_PARSE_BOOL;
}
namespace
{
template <typename T>
T stringToNumber(const String & str)
{
if constexpr (std::is_same_v<T, bool>)
{
if (str == "0")
return false;
if (str == "1")
return true;
if (boost::iequals(str, "false"))
return false;
if (boost::iequals(str, "true"))
return true;
throw Exception("Cannot parse bool from string '" + str + "'", ErrorCodes::CANNOT_PARSE_BOOL);
}
else
return parseWithSizeSuffix<T>(str);
}
template <typename T>
T fieldToNumber(const Field & f)
{
if (f.getType() == Field::Types::String)
return stringToNumber<T>(f.get<const String &>());
else
return applyVisitor(FieldVisitorConvertToNumber<T>(), f);
}
}
template <typename T>
SettingFieldNumber<T>::SettingFieldNumber(const Field & f) : SettingFieldNumber(fieldToNumber<T>(f))
{
}
template <typename T>
SettingFieldNumber<T> & SettingFieldNumber<T>::operator=(const Field & f)
{
*this = fieldToNumber<T>(f);
return *this;
}
template <typename T>
String SettingFieldNumber<T>::toString() const
{
return ::DB::toString(value);
}
template <typename T>
void SettingFieldNumber<T>::parseFromString(const String & str)
{
*this = stringToNumber<T>(str);
}
template <typename T>
void SettingFieldNumber<T>::writeBinary(WriteBuffer & out) const
{
if constexpr (is_integral_v<T> && is_unsigned_v<T>)
writeVarUInt(static_cast<UInt64>(value), out);
else if constexpr (is_integral_v<T> && is_signed_v<T>)
writeVarInt(static_cast<Int64>(value), out);
else
{
static_assert(std::is_floating_point_v<T>);
writeStringBinary(::DB::toString(value), out);
}
}
template <typename T>
void SettingFieldNumber<T>::readBinary(ReadBuffer & in)
{
if constexpr (is_integral_v<T> && is_unsigned_v<T>)
{
UInt64 x;
readVarUInt(x, in);
*this = static_cast<T>(x);
}
else if constexpr (is_integral_v<T> && is_signed_v<T>)
{
Int64 x;
readVarInt(x, in);
*this = static_cast<T>(value);
}
else
{
static_assert(std::is_floating_point_v<T>);
String str;
readStringBinary(str, in);
*this = ::DB::parseFromString<T>(str);
}
}
template struct SettingFieldNumber<UInt64>;
template struct SettingFieldNumber<Int64>;
template struct SettingFieldNumber<float>;
template struct SettingFieldNumber<bool>;
namespace
{
UInt64 stringToMaxThreads(const String & str)
{
if (startsWith(str, "auto"))
return 0;
return parseFromString<UInt64>(str);
}
UInt64 fieldToMaxThreads(const Field & f)
{
if (f.getType() == Field::Types::String)
return stringToMaxThreads(f.get<const String &>());
else
return applyVisitor(FieldVisitorConvertToNumber<UInt64>(), f);
}
}
SettingFieldMaxThreads::SettingFieldMaxThreads(const Field & f) : SettingFieldMaxThreads(fieldToMaxThreads(f))
{
}
SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f)
{
*this = fieldToMaxThreads(f);
return *this;
}
String SettingFieldMaxThreads::toString() const
{
if (is_auto)
return "'auto(" + ::DB::toString(value) + ")'";
else
return ::DB::toString(value);
}
void SettingFieldMaxThreads::parseFromString(const String & str)
{
*this = stringToMaxThreads(str);
}
void SettingFieldMaxThreads::writeBinary(WriteBuffer & out) const
{
writeVarUInt(is_auto ? 0 : value, out);
}
void SettingFieldMaxThreads::readBinary(ReadBuffer & in)
{
UInt64 x = 0;
readVarUInt(x, in);
*this = x;
}
UInt64 SettingFieldMaxThreads::getAuto()
{
return getNumberOfPhysicalCPUCores();
}
template <SettingFieldTimespanUnit unit_>
SettingFieldTimespan<unit_>::SettingFieldTimespan(const Field & f) : SettingFieldTimespan(fieldToNumber<UInt64>(f))
{
}
template <SettingFieldTimespanUnit unit_>
SettingFieldTimespan<unit_> & SettingFieldTimespan<unit_>::operator=(const Field & f)
{
*this = fieldToNumber<UInt64>(f);
return *this;
}
template <SettingFieldTimespanUnit unit_>
String SettingFieldTimespan<unit_>::toString() const
{
return ::DB::toString(operator UInt64());
}
template <SettingFieldTimespanUnit unit_>
void SettingFieldTimespan<unit_>::parseFromString(const String & str)
{
*this = stringToNumber<UInt64>(str);
}
template <SettingFieldTimespanUnit unit_>
void SettingFieldTimespan<unit_>::writeBinary(WriteBuffer & out) const
{
auto num_units = operator UInt64();
writeVarUInt(num_units, out);
}
template <SettingFieldTimespanUnit unit_>
void SettingFieldTimespan<unit_>::readBinary(ReadBuffer & in)
{
UInt64 num_units = 0;
readVarUInt(num_units, in);
*this = num_units;
}
template struct SettingFieldTimespan<SettingFieldTimespanUnit::Second>;
template struct SettingFieldTimespan<SettingFieldTimespanUnit::Millisecond>;
void SettingFieldString::writeBinary(WriteBuffer & out) const
{
writeStringBinary(value, out);
}
void SettingFieldString::readBinary(ReadBuffer & in)
{
String str;
readStringBinary(str, in);
*this = std::move(str);
}
namespace
{
char stringToChar(const String & str)
{
if (str.size() > 1)
throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH);
if (str.empty())
return '\0';
return str[0];
}
char fieldToChar(const Field & f)
{
return stringToChar(f.safeGet<const String &>());
}
}
SettingFieldChar::SettingFieldChar(const Field & f) : SettingFieldChar(fieldToChar(f))
{
}
SettingFieldChar & SettingFieldChar::operator =(const Field & f)
{
*this = fieldToChar(f);
return *this;
}
void SettingFieldChar::parseFromString(const String & str)
{
*this = stringToChar(str);
}
void SettingFieldChar::writeBinary(WriteBuffer & out) const
{
writeStringBinary(toString(), out);
}
void SettingFieldChar::readBinary(ReadBuffer & in)
{
String str;
readStringBinary(str, in);
*this = stringToChar(str);
}
void SettingFieldURI::writeBinary(WriteBuffer & out) const
{
writeStringBinary(value.toString(), out);
}
void SettingFieldURI::readBinary(ReadBuffer & in)
{
String str;
readStringBinary(str, in);
*this = Poco::URI{str};
}
void SettingFieldEnumHelpers::writeBinary(const std::string_view & str, WriteBuffer & out)
{
writeStringBinary(str, out);
}
String SettingFieldEnumHelpers::readBinary(ReadBuffer & in)
{
String str;
readStringBinary(str, in);
return str;
}
String SettingFieldCustom::toString() const
{
return value.dump();
}
void SettingFieldCustom::parseFromString(const String & str)
{
*this = Field::restoreFromDump(str);
}
void SettingFieldCustom::writeBinary(WriteBuffer & out) const
{
writeStringBinary(toString(), out);
}
void SettingFieldCustom::readBinary(ReadBuffer & in)
{
String str;
readStringBinary(str, in);
parseFromString(str);
}
}

349
src/Core/SettingsFields.h Normal file
View File

@ -0,0 +1,349 @@
#pragma once
#include <Poco/Timespan.h>
#include <Poco/URI.h>
#include <Core/Types.h>
#include <Core/Field.h>
#include <boost/range/adaptor/map.hpp>
#include <chrono>
#include <unordered_map>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
class ReadBuffer;
class WriteBuffer;
/** One setting for any type.
* Stores a value within itself, as well as a flag - whether the value was changed.
* This is done so that you can send to the remote servers only changed settings (or explicitly specified in the config) values.
* That is, if the configuration was not specified in the config and was not dynamically changed, it is not sent to the remote server,
* and the remote server will use its default value.
*/
template <typename T>
struct SettingFieldNumber
{
using Type = T;
Type value;
bool changed = false;
explicit SettingFieldNumber(Type x = 0) : value(x) {}
explicit SettingFieldNumber(const Field & f);
SettingFieldNumber & operator=(Type x) { value = x; changed = true; return *this; }
SettingFieldNumber & operator=(const Field & f);
operator Type() const { return value; }
explicit operator Field() const { return value; }
String toString() const;
void parseFromString(const String & str);
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
};
using SettingFieldUInt64 = SettingFieldNumber<UInt64>;
using SettingFieldInt64 = SettingFieldNumber<Int64>;
using SettingFieldFloat = SettingFieldNumber<float>;
using SettingFieldBool = SettingFieldNumber<bool>;
/** Unlike SettingFieldUInt64, supports the value of 'auto' - the number of processor cores without taking into account SMT.
* A value of 0 is also treated as auto.
* When serializing, `auto` is written in the same way as 0.
*/
struct SettingFieldMaxThreads
{
bool is_auto;
UInt64 value;
bool changed = false;
explicit SettingFieldMaxThreads(UInt64 x = 0) : is_auto(!x), value(is_auto ? getAuto() : x) {}
explicit SettingFieldMaxThreads(const Field & f);
SettingFieldMaxThreads & operator=(UInt64 x) { is_auto = !x; value = is_auto ? getAuto() : x; changed = true; return *this; }
SettingFieldMaxThreads & operator=(const Field & f);
operator UInt64() const { return value; }
explicit operator Field() const { return value; }
/// Writes "auto(<number>)" instead of simple "<number>" if `is_auto==true`.
String toString() const;
void parseFromString(const String & str);
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
private:
static UInt64 getAuto();
};
enum class SettingFieldTimespanUnit { Millisecond, Second };
template <SettingFieldTimespanUnit unit_>
struct SettingFieldTimespan
{
using Unit = SettingFieldTimespanUnit;
static constexpr Unit unit = unit_;
static constexpr UInt64 microseconds_per_unit = (unit == SettingFieldTimespanUnit::Millisecond) ? 1000 : 1000000;
Poco::Timespan value;
bool changed = false;
explicit SettingFieldTimespan(const Poco::Timespan & x = {}) : value(x) {}
template <class Rep, class Period = std::ratio<1>>
explicit SettingFieldTimespan(const std::chrono::duration<Rep, Period> & x)
: SettingFieldTimespan(Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(std::chrono::duration_cast<std::chrono::microseconds>(x).count())}) {}
explicit SettingFieldTimespan(UInt64 x) : SettingFieldTimespan(Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(x * microseconds_per_unit)}) {}
explicit SettingFieldTimespan(const Field & f);
SettingFieldTimespan & operator =(const Poco::Timespan & x) { value = x; changed = true; return *this; }
template <class Rep, class Period = std::ratio<1>>
SettingFieldTimespan & operator =(const std::chrono::duration<Rep, Period> & x) { *this = Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(std::chrono::duration_cast<std::chrono::microseconds>(x).count())}; return *this; }
SettingFieldTimespan & operator =(UInt64 x) { *this = Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(x * microseconds_per_unit)}; return *this; }
SettingFieldTimespan & operator =(const Field & f);
operator Poco::Timespan() const { return value; }
template <class Rep, class Period = std::ratio<1>>
operator std::chrono::duration<Rep, Period>() const { return std::chrono::duration_cast<std::chrono::duration<Rep, Period>>(std::chrono::microseconds(value.totalMicroseconds())); }
explicit operator UInt64() const { return value.totalMicroseconds() / microseconds_per_unit; }
explicit operator Field() const { return operator UInt64(); }
Poco::Timespan::TimeDiff totalMicroseconds() const { return value.totalMicroseconds(); }
Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); }
Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); }
String toString() const;
void parseFromString(const String & str);
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
};
using SettingFieldSeconds = SettingFieldTimespan<SettingFieldTimespanUnit::Second>;
using SettingFieldMilliseconds = SettingFieldTimespan<SettingFieldTimespanUnit::Millisecond>;
struct SettingFieldString
{
String value;
bool changed = false;
explicit SettingFieldString(const std::string_view & str = {}) : value(str) {}
explicit SettingFieldString(const String & str) : SettingFieldString(std::string_view{str}) {}
explicit SettingFieldString(String && str) : value(std::move(str)) {}
explicit SettingFieldString(const char * str) : SettingFieldString(std::string_view{str}) {}
explicit SettingFieldString(const Field & f) : SettingFieldString(f.safeGet<const String &>()) {}
SettingFieldString & operator =(const std::string_view & str) { value = str; changed = true; return *this; }
SettingFieldString & operator =(const String & str) { *this = std::string_view{str}; return *this; }
SettingFieldString & operator =(String && str) { value = std::move(str); changed = true; return *this; }
SettingFieldString & operator =(const char * str) { *this = std::string_view{str}; return *this; }
SettingFieldString & operator =(const Field & f) { *this = f.safeGet<const String &>(); return *this; }
operator const String &() const { return value; }
explicit operator Field() const { return value; }
const String & toString() const { return value; }
void parseFromString(const String & str) { *this = str; }
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
};
struct SettingFieldChar
{
public:
char value;
bool changed = false;
explicit SettingFieldChar(char c = '\0') : value(c) {}
explicit SettingFieldChar(const Field & f);
SettingFieldChar & operator =(char c) { value = c; changed = true; return *this; }
SettingFieldChar & operator =(const Field & f);
operator char() const { return value; }
explicit operator Field() const { return toString(); }
String toString() const { return String(&value, 1); }
void parseFromString(const String & str);
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
};
struct SettingFieldURI
{
Poco::URI value;
bool changed = false;
explicit SettingFieldURI(const Poco::URI & uri = {}) : value(uri) {}
explicit SettingFieldURI(const String & str) : SettingFieldURI(Poco::URI{str}) {}
explicit SettingFieldURI(const char * str) : SettingFieldURI(Poco::URI{str}) {}
explicit SettingFieldURI(const Field & f) : SettingFieldURI(f.safeGet<String>()) {}
SettingFieldURI & operator =(const Poco::URI & x) { value = x; changed = true; return *this; }
SettingFieldURI & operator =(const String & str) { *this = Poco::URI{str}; return *this; }
SettingFieldURI & operator =(const char * str) { *this = Poco::URI{str}; return *this; }
SettingFieldURI & operator =(const Field & f) { *this = f.safeGet<const String &>(); return *this; }
operator const Poco::URI &() const { return value; }
explicit operator String() const { return toString(); }
explicit operator Field() const { return toString(); }
String toString() const { return value.toString(); }
void parseFromString(const String & str) { *this = str; }
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
};
/** Template class to define enum-based settings.
* Example of usage:
*
* mysettings.h:
* enum Gender { Male, Female };
* DECLARE_SETTING_ENUM(SettingFieldGender, Gender)
*
* mysettings.cpp:
* IMPLEMENT_SETTING_ENUM(SettingFieldGender, ErrorCodes::BAD_ARGUMENTS,
* {{"Male", Gender::Male}, {"Female", Gender::Female}})
*/
template <typename EnumT, typename Traits>
struct SettingFieldEnum
{
using EnumType = EnumT;
EnumType value;
bool changed = false;
explicit SettingFieldEnum(EnumType x = EnumType{0}) : value(x) {}
explicit SettingFieldEnum(const Field & f) : SettingFieldEnum(Traits::fromString(f.safeGet<const String &>())) {}
SettingFieldEnum & operator =(EnumType x) { value = x; changed = true; return *this; }
SettingFieldEnum & operator =(const Field & f) { *this = Traits::fromString(f.safeGet<const String &>()); return *this; }
operator EnumType() const { return value; }
explicit operator Field() const { return toString(); }
String toString() const { return Traits::toString(value); }
void parseFromString(const String & str) { *this = Traits::fromString(str); }
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
};
struct SettingFieldEnumHelpers
{
static void writeBinary(const std::string_view & str, WriteBuffer & out);
static String readBinary(ReadBuffer & in);
};
template <typename EnumT, typename Traits>
void SettingFieldEnum<EnumT, Traits>::writeBinary(WriteBuffer & out) const
{
SettingFieldEnumHelpers::writeBinary(toString(), out);
}
template <typename EnumT, typename Traits>
void SettingFieldEnum<EnumT, Traits>::readBinary(ReadBuffer & in)
{
*this = Traits::fromString(SettingFieldEnumHelpers::readBinary(in));
}
#define DECLARE_SETTING_ENUM(ENUM_TYPE) \
DECLARE_SETTING_ENUM_WITH_RENAME(ENUM_TYPE, ENUM_TYPE)
#define IMPLEMENT_SETTING_ENUM(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \
IMPLEMENT_SETTING_ENUM_WITH_RENAME(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, __VA_ARGS__)
#define DECLARE_SETTING_ENUM_WITH_RENAME(NEW_NAME, ENUM_TYPE) \
struct SettingField##NEW_NAME##Traits \
{ \
using EnumType = ENUM_TYPE; \
static const String & toString(EnumType value); \
static EnumType fromString(const std::string_view & str); \
}; \
\
using SettingField##NEW_NAME = SettingFieldEnum<ENUM_TYPE, SettingField##NEW_NAME##Traits>;
#define IMPLEMENT_SETTING_ENUM_WITH_RENAME(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \
const String & SettingField##NEW_NAME##Traits::toString(typename SettingField##NEW_NAME::EnumType value) \
{ \
static const std::unordered_map<EnumType, String> map = [] { \
std::unordered_map<EnumType, String> res; \
constexpr std::pair<const char *, EnumType> pairs[] = __VA_ARGS__; \
for (const auto & [name, val] : pairs) \
res.emplace(val, name); \
return res; \
}(); \
auto it = map.find(value); \
if (it != map.end()) \
return it->second; \
throw Exception( \
"Unexpected value of " #NEW_NAME ":" + std::to_string(std::underlying_type<EnumType>::type(value)), \
ERROR_CODE_FOR_UNEXPECTED_NAME); \
} \
\
typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(const std::string_view & str) \
{ \
static const std::unordered_map<std::string_view, EnumType> map = [] { \
std::unordered_map<std::string_view, EnumType> res; \
constexpr std::pair<const char *, EnumType> pairs[] = __VA_ARGS__; \
for (const auto & [name, val] : pairs) \
res.emplace(name, val); \
return res; \
}(); \
auto it = map.find(str); \
if (it != map.end()) \
return it->second; \
String msg = "Unexpected value of " #NEW_NAME ": '" + String{str} + "'. Must be one of ["; \
bool need_comma = false; \
for (auto & name : map | boost::adaptors::map_keys) \
{ \
if (std::exchange(need_comma, true)) \
msg += ", "; \
msg += "'" + String{name} + "'"; \
} \
msg += "]"; \
throw Exception(msg, ERROR_CODE_FOR_UNEXPECTED_NAME); \
}
/// Can keep a value of any type. Used for user-defined settings.
struct SettingFieldCustom
{
Field value;
bool changed = false;
explicit SettingFieldCustom(const Field & f = {}) : value(f) {}
SettingFieldCustom & operator =(const Field & f) { value = f; changed = true; return *this; }
explicit operator Field() const { return value; }
String toString() const;
void parseFromString(const String & str);
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
};
}

View File

@ -5,7 +5,7 @@
#include <cstddef>
#include <string>
#include <Core/Field.h>
#include <Core/SettingsCollection.h>
#include <Core/SettingsEnums.h>
class Collator;

View File

@ -8,6 +8,7 @@ PEERDIR(
SRCS(
BackgroundSchedulePool.cpp
BaseSettings.cpp
Block.cpp
BlockInfo.cpp
ColumnWithTypeAndName.cpp
@ -19,7 +20,8 @@ SRCS(
PostgreSQLProtocol.cpp
NamesAndTypes.cpp
Settings.cpp
SettingsCollection.cpp
SettingsEnums.cpp
SettingsFields.cpp
SortDescription.cpp
)

View File

@ -91,8 +91,7 @@ TTLBlockInputStream::TTLBlockInputStream(
const Settings & settings = storage.global_context.getSettingsRef();
Aggregator::Params params(header, keys, aggregates,
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
SettingUInt64(0), SettingUInt64(0),
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0,
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
storage.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
aggregator = std::make_unique<Aggregator>(params);

View File

@ -4,7 +4,6 @@
#if USE_MYSQL
# include <string>
# include <Core/SettingsCollection.h>
# include <DataTypes/DataTypeDateTime.h>
# include <DataTypes/DataTypeNullable.h>
# include <DataTypes/DataTypeString.h>
@ -42,7 +41,7 @@ namespace ErrorCodes
constexpr static const auto suffix = ".remove_flag";
static constexpr const std::chrono::seconds cleaner_sleep_time{30};
static const SettingSeconds lock_acquire_timeout{10};
static const std::chrono::seconds lock_acquire_timeout{10};
static String toQueryStringWithQuote(const std::vector<String> & quote_list)
{

View File

@ -426,7 +426,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
}
/// Read and parse metadata in parallel
ThreadPool pool(SettingMaxThreads().getAutoValue());
ThreadPool pool;
for (const auto & file : metadata_files)
{
pool.scheduleOrThrowOnError([&]()

View File

@ -152,7 +152,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
std::atomic<size_t> tables_processed{0};
std::atomic<size_t> dictionaries_processed{0};
ThreadPool pool(SettingMaxThreads().getAutoValue());
ThreadPool pool;
/// Attach tables.
for (const auto & name_with_query : file_names)

View File

@ -0,0 +1,67 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/FieldToDataType.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/Context.h>
#include <Core/Field.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/// Get the value of a setting.
class FunctionGetSetting : public IFunction
{
public:
static constexpr auto name = "getSetting";
static FunctionPtr create(const Context & context_) { return std::make_shared<FunctionGetSetting>(context_); }
explicit FunctionGetSetting(const Context & context_) : context(context_) {}
String getName() const override { return name; }
bool isDeterministic() const override { return false; }
size_t getNumberOfArguments() const override { return 1; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (!isString(arguments[0].type))
throw Exception{"The argument of function " + String{name} + " should be a constant string with the name of a setting",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
const auto * column = arguments[0].column.get();
if (!column || !checkAndGetColumnConstStringOrFixedString(column))
throw Exception{"The argument of function " + String{name} + " should be a constant string with the name of a setting",
ErrorCodes::ILLEGAL_COLUMN};
std::string_view setting_name{column->getDataAt(0)};
value = context.getSettingsRef().get(setting_name);
DataTypePtr type = applyVisitor(FieldToDataType{}, value);
value = convertFieldToType(value, *type);
return type;
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, value);
}
private:
mutable Field value;
const Context & context;
};
void registerFunctionGetSetting(FunctionFactory & factory)
{
factory.registerFunction<FunctionGetSetting>();
}
}

View File

@ -56,6 +56,7 @@ void registerFunctionBasename(FunctionFactory &);
void registerFunctionTransform(FunctionFactory &);
void registerFunctionGetMacro(FunctionFactory &);
void registerFunctionGetScalar(FunctionFactory &);
void registerFunctionGetSetting(FunctionFactory &);
void registerFunctionIsConstant(FunctionFactory &);
void registerFunctionGlobalVariable(FunctionFactory &);
void registerFunctionHasThreadFuzzer(FunctionFactory &);
@ -116,6 +117,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
registerFunctionTransform(factory);
registerFunctionGetMacro(factory);
registerFunctionGetScalar(factory);
registerFunctionGetSetting(factory);
registerFunctionIsConstant(factory);
registerFunctionGlobalVariable(factory);
registerFunctionHasThreadFuzzer(factory);

View File

@ -203,6 +203,7 @@ SRCS(
geoToH3.cpp
getMacro.cpp
getScalar.cpp
getSetting.cpp
getSizeOfEnumType.cpp
globalVariable.cpp
greatCircleDistance.cpp

View File

@ -638,7 +638,7 @@ template <typename T>
inline T parse(const char * data, size_t size);
template <typename T>
inline T parseFromString(const String & str)
inline T parseFromString(const std::string_view & str)
{
return parse<T>(str.data(), str.size());
}
@ -861,6 +861,13 @@ inline void readQuoted(LocalDateTime & x, ReadBuffer & buf)
assertChar('\'', buf);
}
inline void readQuoted(UUID & x, ReadBuffer & buf)
{
assertChar('\'', buf);
readUUIDText(x, buf);
assertChar('\'', buf);
}
/// Same as above, but in double quotes.
template <typename T>
@ -1096,7 +1103,7 @@ inline T parseWithSizeSuffix(const char * data, size_t size)
}
template <typename T>
inline T parseWithSizeSuffix(const String & s)
inline T parseWithSizeSuffix(const std::string_view & s)
{
return parseWithSizeSuffix<T>(s.data(), s.size());
}

View File

@ -44,14 +44,14 @@ protected:
UInt64 redirects { 0 };
Poco::URI initial_uri;
const ConnectionTimeouts & timeouts;
SettingUInt64 max_redirects;
UInt64 max_redirects;
public:
virtual void buildNewSession(const Poco::URI & uri) = 0;
explicit UpdatableSessionBase(const Poco::URI uri,
const ConnectionTimeouts & timeouts_,
SettingUInt64 max_redirects_)
UInt64 max_redirects_)
: initial_uri { uri }
, timeouts { timeouts_ }
, max_redirects { max_redirects_ }
@ -228,7 +228,7 @@ class UpdatableSession : public UpdatableSessionBase<HTTPSessionPtr>
public:
explicit UpdatableSession(const Poco::URI uri,
const ConnectionTimeouts & timeouts_,
const SettingUInt64 max_redirects_)
const UInt64 max_redirects_)
: Parent(uri, timeouts_, max_redirects_)
{
session = makeHTTPSession(initial_uri, timeouts);
@ -249,7 +249,7 @@ public:
const std::string & method_,
OutStreamCallback out_stream_callback_,
const ConnectionTimeouts & timeouts,
const SettingUInt64 max_redirects = 0,
const UInt64 max_redirects = 0,
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const HTTPHeaderEntries & http_header_entries_ = {},
@ -269,7 +269,7 @@ private:
public:
explicit UpdatablePooledSession(const Poco::URI uri,
const ConnectionTimeouts & timeouts_,
const SettingUInt64 max_redirects_,
const UInt64 max_redirects_,
size_t per_endpoint_pool_size_)
: Parent(uri, timeouts_, max_redirects_)
, per_endpoint_pool_size { per_endpoint_pool_size_ }
@ -294,7 +294,7 @@ public:
const ConnectionTimeouts & timeouts_ = {},
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const SettingUInt64 max_redirects = 0,
const UInt64 max_redirects = 0,
size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT)
: Parent(std::make_shared<UpdatablePooledSession>(uri_, timeouts_, max_redirects, max_connections_per_endpoint),
uri_,

View File

@ -345,8 +345,6 @@ ReturnType parseDateTimeBestEffortImpl(
}
else
return on_error("Cannot read DateTime: unexpected number of decimal digits after day of month: " + toString(num_digits), ErrorCodes::CANNOT_PARSE_DATETIME);
}
if (month > 12)

View File

@ -968,7 +968,7 @@ void Context::setSetting(const StringRef & name, const String & value)
setProfile(value);
return;
}
settings.set(name, value);
settings.set(std::string_view{name}, value);
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
@ -983,7 +983,7 @@ void Context::setSetting(const StringRef & name, const Field & value)
setProfile(value.safeGet<String>());
return;
}
settings.set(name, value);
settings.set(std::string_view{name}, value);
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
@ -1025,11 +1025,10 @@ void Context::checkSettingsConstraints(const SettingsChanges & changes) const
settings_constraints->check(settings, changes);
}
void Context::clampToSettingsConstraints(SettingChange & change) const
void Context::checkSettingsConstraints(SettingsChanges & changes) const
{
if (auto settings_constraints = getSettingsConstraints())
settings_constraints->clamp(settings, change);
settings_constraints->check(settings, changes);
}
void Context::clampToSettingsConstraints(SettingsChanges & changes) const
@ -1038,7 +1037,6 @@ void Context::clampToSettingsConstraints(SettingsChanges & changes) const
settings_constraints->clamp(settings, changes);
}
std::shared_ptr<const SettingsConstraints> Context::getSettingsConstraints() const
{
return getAccess()->getSettingsConstraints();

View File

@ -377,7 +377,7 @@ public:
/// Checks the constraints.
void checkSettingsConstraints(const SettingChange & change) const;
void checkSettingsConstraints(const SettingsChanges & changes) const;
void clampToSettingsConstraints(SettingChange & change) const;
void checkSettingsConstraints(SettingsChanges & changes) const;
void clampToSettingsConstraints(SettingsChanges & changes) const;
/// Returns the current constraints (can return null).

View File

@ -560,7 +560,7 @@ void DatabaseCatalog::loadMarkedAsDroppedTables()
dropped_metadata.emplace(std::move(full_path), std::move(dropped_id));
}
ThreadPool pool(SettingMaxThreads().getAutoValue());
ThreadPool pool;
for (const auto & elem : dropped_metadata)
{
pool.scheduleOrThrowOnError([&]()

View File

@ -53,7 +53,7 @@ struct NonGlobalTableData
private:
void renameIfNeeded(ASTPtr & database_and_table)
{
const SettingDistributedProductMode distributed_product_mode = context.getSettingsRef().distributed_product_mode;
const DistributedProductMode distributed_product_mode = context.getSettingsRef().distributed_product_mode;
StoragePtr storage = tryGetTable(database_and_table, context);
if (!storage || !checker.hasAtLeastTwoShards(*storage))

View File

@ -1527,8 +1527,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific
const Settings & settings = context->getSettingsRef();
Aggregator::Params params(header_before_transform, keys, query_analyzer->aggregates(),
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
SettingUInt64(0), SettingUInt64(0),
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0,
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);

View File

@ -2,7 +2,6 @@
#include <Interpreters/SystemLog.h>
#include <Interpreters/ClientInfo.h>
#include <Core/SettingsCollection.h>
namespace ProfileEvents

View File

@ -2,7 +2,7 @@
#include <Core/Names.h>
#include <Core/NamesAndTypes.h>
#include <Core/SettingsCollection.h>
#include <Core/SettingsEnums.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/IJoin.h>
#include <Interpreters/asof.h>

View File

@ -8,7 +8,7 @@ namespace DB
class ASTSelectQuery;
struct SettingChange;
using SettingsChanges = std::vector<SettingChange>;
class SettingsChanges;
/// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query:
/// (since settings after FORMAT parsed separatelly not in the ParserSelectQuery but in ParserQueryWithOutput)

View File

@ -880,10 +880,10 @@ void TCPHandler::receiveQuery()
/// Per query settings are also passed via TCP.
/// We need to check them before applying due to they can violate the settings constraints.
auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS
: SettingsBinaryFormat::OLD;
auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS
: SettingsWriteFormat::BINARY;
Settings passed_settings;
passed_settings.deserialize(*in, settings_format);
passed_settings.read(*in, settings_format);
auto settings_changes = passed_settings.changes();
if (client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
{
@ -925,9 +925,9 @@ void TCPHandler::receiveUnexpectedQuery()
skip_client_info.read(*in, client_revision);
Settings skip_settings;
auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS
: SettingsBinaryFormat::OLD;
skip_settings.deserialize(*in, settings_format);
auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS
: SettingsWriteFormat::BINARY;
skip_settings.read(*in, settings_format);
readVarUInt(skip_uint_64, *in);
readVarUInt(skip_uint_64, *in);

View File

@ -368,7 +368,7 @@ void StorageDistributedDirectoryMonitor::readHeader(
}
readStringBinary(insert_query, header_buf);
insert_settings.deserialize(header_buf);
insert_settings.read(header_buf);
if (header_buf.hasPendingData())
client_info.read(header_buf, initiator_revision);
@ -382,7 +382,7 @@ void StorageDistributedDirectoryMonitor::readHeader(
if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT)
{
insert_settings.deserialize(in, SettingsBinaryFormat::OLD);
insert_settings.read(in, SettingsWriteFormat::BINARY);
readStringBinary(insert_query, in);
return;
}

View File

@ -591,7 +591,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std::
WriteBufferFromOwnString header_buf;
writeVarUInt(ClickHouseRevision::get(), header_buf);
writeStringBinary(query_string, header_buf);
context.getSettingsRef().serialize(header_buf);
context.getSettingsRef().write(header_buf);
context.getClientInfo().write(header_buf, ClickHouseRevision::get());
/// Add new fields here, for example:

View File

@ -29,22 +29,22 @@ bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadata
}
RWLockImpl::LockHolder IStorage::tryLockTimed(
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const
{
auto lock_holder = rwlock->getLock(type, query_id, std::chrono::milliseconds(acquire_timeout.totalMilliseconds()));
auto lock_holder = rwlock->getLock(type, query_id, acquire_timeout);
if (!lock_holder)
{
const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE";
throw Exception(
type_str + " locking attempt on \"" + getStorageID().getFullTableName() +
"\" has timed out! (" + toString(acquire_timeout.totalMilliseconds()) + "ms) "
"\" has timed out! (" + std::to_string(acquire_timeout.count()) + "ms) "
"Possible deadlock avoided. Client should retry.",
ErrorCodes::DEADLOCK_AVOIDED);
}
return lock_holder;
}
TableLockHolder IStorage::lockForShare(const String & query_id, const SettingSeconds & acquire_timeout)
TableLockHolder IStorage::lockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout)
{
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
@ -54,7 +54,7 @@ TableLockHolder IStorage::lockForShare(const String & query_id, const SettingSec
return result;
}
TableLockHolder IStorage::lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout)
TableLockHolder IStorage::lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout)
{
TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);
@ -65,7 +65,7 @@ TableLockHolder IStorage::lockForAlter(const String & query_id, const SettingSec
}
TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout)
TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout)
{
TableExclusiveLockHolder result;
result.alter_lock = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);

View File

@ -37,8 +37,6 @@ using StorageActionBlockType = size_t;
class ASTCreateQuery;
struct Settings;
struct SettingChange;
using SettingsChanges = std::vector<SettingChange>;
class AlterCommands;
class MutationCommands;
@ -185,18 +183,18 @@ private:
MultiVersionStorageMetadataPtr metadata;
private:
RWLockImpl::LockHolder tryLockTimed(
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const;
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const;
public:
/// Lock table for share. This lock must be acuqired if you want to be sure,
/// that table will be not dropped while you holding this lock. It's used in
/// variety of cases starting from SELECT queries to background merges in
/// MergeTree.
TableLockHolder lockForShare(const String & query_id, const SettingSeconds & acquire_timeout);
TableLockHolder lockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout);
/// Lock table for alter. This lock must be acuqired in ALTER queries to be
/// sure, that we execute only one simultaneous alter. Doesn't affect share lock.
TableLockHolder lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout);
TableLockHolder lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout);
/// Lock table exclusively. This lock must be acuired if you want to be
/// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something
@ -205,7 +203,7 @@ public:
///
/// NOTE: You have to be 100% sure that you need this lock. It's extremely
/// heavyweight and makes table irresponsive.
TableExclusiveLockHolder lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout);
TableExclusiveLockHolder lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout);
/** Returns stage to which query is going to be processed in read() function.
* (Normally, the function only reads the columns from the list, but in other cases,

View File

@ -3,7 +3,6 @@
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/Exception.h>
#include <Core/SettingsCollectionImpl.h>
namespace DB
@ -15,7 +14,7 @@ namespace ErrorCodes
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_COLLECTION(KafkaSettings, LIST_OF_KAFKA_SETTINGS)
IMPLEMENT_SETTINGS_TRAITS(KafkaSettingsTraits, LIST_OF_KAFKA_SETTINGS)
void KafkaSettings::loadFromQuery(ASTStorage & storage_def)
{

View File

@ -1,40 +1,34 @@
#pragma once
#include <Core/SettingsCollection.h>
#include <Core/BaseSettings.h>
#include <Core/Settings.h>
namespace DB
{
class ASTStorage;
/** Settings for the Kafka engine.
* Could be loaded from a CREATE TABLE query (SETTINGS clause).
*/
struct KafkaSettings : public SettingsCollection<KafkaSettings>
{
#define KAFKA_RELATED_SETTINGS(M) \
M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \
M(SettingString, kafka_topic_list, "", "A list of Kafka topics.", 0) \
M(SettingString, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \
M(SettingString, kafka_client_id, "", "Client identifier.", 0) \
M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \
M(SettingBool, kafka_commit_every_batch, false, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) \
#define KAFKA_RELATED_SETTINGS(M) \
M(String, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \
M(String, kafka_topic_list, "", "A list of Kafka topics.", 0) \
M(String, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \
M(String, kafka_client_id, "", "Client identifier.", 0) \
M(UInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \
M(Bool, kafka_commit_every_batch, false, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) \
/* default is stream_poll_timeout_ms */ \
M(SettingMilliseconds, kafka_poll_timeout_ms, 0, "Timeout for single poll from Kafka.", 0) \
M(Milliseconds, kafka_poll_timeout_ms, 0, "Timeout for single poll from Kafka.", 0) \
/* default is min(max_block_size, kafka_max_block_size)*/ \
M(SettingUInt64, kafka_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \
M(UInt64, kafka_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \
/* default is = min_insert_block_size / kafka_num_consumers */ \
M(SettingUInt64, kafka_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \
M(UInt64, kafka_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \
/* default is stream_flush_interval_ms */ \
M(SettingMilliseconds, kafka_flush_interval_ms, 0, "Timeout for flushing data from Kafka.", 0) \
M(Milliseconds, kafka_flush_interval_ms, 0, "Timeout for flushing data from Kafka.", 0) \
/* those are mapped to format factory settings */ \
M(SettingString, kafka_format, "", "The message format for Kafka engine.", 0) \
M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \
M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \
M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0)
M(String, kafka_format, "", "The message format for Kafka engine.", 0) \
M(Char, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \
M(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \
M(UInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0)
/** TODO: */
/* https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md */
@ -44,8 +38,14 @@ struct KafkaSettings : public SettingsCollection<KafkaSettings>
KAFKA_RELATED_SETTINGS(M) \
FORMAT_FACTORY_SETTINGS(M)
DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS)
DECLARE_SETTINGS_TRAITS(KafkaSettingsTraits, LIST_OF_KAFKA_SETTINGS)
/** Settings for the Kafka engine.
* Could be loaded from a CREATE TABLE query (SETTINGS clause).
*/
struct KafkaSettings : public BaseSettings<KafkaSettingsTraits>
{
void loadFromQuery(ASTStorage & storage_def);
};

View File

@ -169,12 +169,11 @@ SettingsChanges StorageKafka::createSettingsAdjustments()
if (!schema_name.empty())
result.emplace_back("format_schema", schema_name);
for (auto & it : *kafka_settings)
for (auto setting : *kafka_settings)
{
if (it.isChanged() && it.getName().toString().rfind("kafka_",0) == std::string::npos)
{
result.emplace_back(it.getName().toString(), it.getValueAsString());
}
const auto & name = setting.getName();
if (name.find("kafka_") == std::string::npos)
result.emplace_back(name, setting.getValue());
}
return result;
}
@ -632,8 +631,8 @@ void registerStorageKafka(StorageFactory & factory)
engine_args[(ARG_NUM)-1], \
args.local_context); \
} \
kafka_settings->PAR_NAME.set( \
engine_args[(ARG_NUM)-1]->as<ASTLiteral &>().value);\
kafka_settings->PAR_NAME = \
engine_args[(ARG_NUM)-1]->as<ASTLiteral &>().value; \
} \
}

View File

@ -5,6 +5,7 @@
#include <Storages/Kafka/Buffer_fwd.h>
#include <Storages/Kafka/KafkaSettings.h>
#include <Interpreters/Context.h>
#include <Common/SettingsChanges.h>
#include <Poco/Semaphore.h>
#include <ext/shared_ptr_helper.h>

View File

@ -103,7 +103,6 @@ namespace ErrorCodes
extern const int BAD_TTL_EXPRESSION;
extern const int INCORRECT_FILE_NAME;
extern const int BAD_DATA_PART_NAME;
extern const int UNKNOWN_SETTING;
extern const int READONLY_SETTING;
extern const int ABORTED;
extern const int UNKNOWN_PART_TYPE;
@ -1475,24 +1474,19 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
const auto & new_changes = new_metadata.settings_changes->as<const ASTSetQuery &>().changes;
for (const auto & changed_setting : new_changes)
{
if (MergeTreeSettings::findIndex(changed_setting.name) == MergeTreeSettings::npos)
throw Exception{"Storage '" + getName() + "' doesn't have setting '" + changed_setting.name + "'",
ErrorCodes::UNKNOWN_SETTING};
const auto & setting_name = changed_setting.name;
const auto & new_value = changed_setting.value;
MergeTreeSettings::checkCanSet(setting_name, new_value);
const Field * current_value = current_changes.tryGet(setting_name);
auto comparator = [&changed_setting](const auto & change) { return change.name == changed_setting.name; };
auto current_setting_it
= std::find_if(current_changes.begin(), current_changes.end(), comparator);
if ((current_setting_it == current_changes.end() || *current_setting_it != changed_setting)
&& MergeTreeSettings::isReadonlySetting(changed_setting.name))
if ((!current_value || *current_value != new_value)
&& MergeTreeSettings::isReadonlySetting(setting_name))
{
throw Exception{"Setting '" + changed_setting.name + "' is readonly for storage '" + getName() + "'",
throw Exception{"Setting '" + setting_name + "' is readonly for storage '" + getName() + "'",
ErrorCodes::READONLY_SETTING};
}
if (current_setting_it == current_changes.end()
&& MergeTreeSettings::isPartFormatSetting(changed_setting.name))
if (!current_value && MergeTreeSettings::isPartFormatSetting(setting_name))
{
MergeTreeSettings copy = *getSettings();
copy.applyChange(changed_setting);
@ -1501,8 +1495,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
throw Exception("Can't change settings. Reason: " + reason, ErrorCodes::NOT_IMPLEMENTED);
}
if (changed_setting.name == "storage_policy")
checkStoragePolicy(global_context.getStoragePolicy(changed_setting.value.safeGet<String>()));
if (setting_name == "storage_policy")
checkStoragePolicy(global_context.getStoragePolicy(new_value.safeGet<String>()));
}
}

View File

@ -4,7 +4,6 @@
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/Exception.h>
#include <Core/SettingsCollectionImpl.h>
namespace DB
@ -17,7 +16,7 @@ namespace ErrorCodes
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_COLLECTION(MergeTreeSettings, LIST_OF_MERGE_TREE_SETTINGS)
IMPLEMENT_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS)
void MergeTreeSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
{

View File

@ -1,117 +1,113 @@
#pragma once
#include <Core/Defines.h>
#include <Core/SettingsCollection.h>
#include <Common/SettingsChanges.h>
#include <Core/BaseSettings.h>
namespace Poco
namespace Poco::Util
{
namespace Util
{
class AbstractConfiguration;
}
class AbstractConfiguration;
}
namespace DB
{
class ASTStorage;
struct Settings;
/** Settings for the MergeTree family of engines.
* Could be loaded from config or from a CREATE TABLE query (SETTINGS clause).
*/
struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
{
#define LIST_OF_MERGE_TREE_SETTINGS(M) \
M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \
#define LIST_OF_MERGE_TREE_SETTINGS(M) \
M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \
\
/** Data storing format settings. */ \
M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \
M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \
M(SettingUInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \
M(SettingUInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \
M(SettingBool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \
M(SettingUInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \
M(UInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \
M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \
M(UInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \
M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \
M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \
M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \
\
/** Merge settings. */ \
M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \
M(SettingUInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \
M(SettingUInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \
M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \
M(SettingUInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \
M(SettingUInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \
M(SettingUInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \
M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \
M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \
M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \
M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \
M(UInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \
M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \
M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \
M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \
M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \
M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \
M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \
M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \
M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \
\
/** Inserts settings. */ \
M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \
M(SettingUInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \
M(SettingUInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \
M(SettingUInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \
M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \
M(UInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \
M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \
M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \
\
/** Replication settings. */ \
M(SettingUInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \
M(SettingUInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \
M(SettingUInt64, max_replicated_logs_to_keep, 100, "How many records may be in log, if there is inactive replica.", 0) \
M(SettingUInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \
M(SettingSeconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \
M(SettingUInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \
M(SettingBool, always_fetch_merged_part, 0, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \
M(SettingUInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \
M(SettingUInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \
M(SettingUInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \
M(SettingFloat, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \
M(SettingUInt64, replicated_max_parallel_fetches, 0, "Limit parallel fetches.", 0) \
M(SettingUInt64, replicated_max_parallel_fetches_for_table, 0, "Limit parallel fetches for one table.", 0) \
M(SettingUInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).", 0) \
M(SettingUInt64, replicated_max_parallel_sends, 0, "Limit parallel sends.", 0) \
M(SettingUInt64, replicated_max_parallel_sends_for_table, 0, "Limit parallel sends for one table.", 0) \
M(SettingBool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \
M(SettingSeconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \
M(UInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \
M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \
M(UInt64, max_replicated_logs_to_keep, 100, "How many records may be in log, if there is inactive replica.", 0) \
M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \
M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \
M(UInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \
M(Bool, always_fetch_merged_part, 0, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \
M(UInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \
M(UInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \
M(UInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \
M(Float, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \
M(UInt64, replicated_max_parallel_fetches, 0, "Limit parallel fetches.", 0) \
M(UInt64, replicated_max_parallel_fetches_for_table, 0, "Limit parallel fetches for one table.", 0) \
M(UInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).", 0) \
M(UInt64, replicated_max_parallel_sends, 0, "Limit parallel sends.", 0) \
M(UInt64, replicated_max_parallel_sends_for_table, 0, "Limit parallel sends for one table.", 0) \
M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \
M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \
\
/** Check delay of replicas settings. */ \
M(SettingUInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \
M(SettingUInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.", 0) \
M(SettingUInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \
M(SettingUInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \
M(SettingUInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \
M(SettingUInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \
M(SettingUInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \
M(SettingUInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \
M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \
M(UInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.", 0) \
M(UInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \
M(UInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \
M(UInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \
M(UInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \
M(UInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \
M(UInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \
\
/** Compatibility settings */ \
M(SettingBool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \
M(SettingBool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \
M(SettingBool, use_minimalistic_part_header_in_zookeeper, true, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (<part>/columns and <part>/checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \
M(SettingUInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \
M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \
M(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \
M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \
M(SettingBool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \
M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \
M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity", 0) \
M(SettingMaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \
M(SettingMaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \
M(SettingUInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \
M(SettingString, storage_policy, "default", "Name of storage disk policy", 0) \
M(SettingBool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \
M(Bool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \
M(Bool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \
M(Bool, use_minimalistic_part_header_in_zookeeper, true, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (<part>/columns and <part>/checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \
M(UInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \
M(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \
M(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \
M(Int64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \
M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \
M(Bool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \
M(Bool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity", 0) \
M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \
M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \
M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \
M(String, storage_policy, "default", "Name of storage disk policy", 0) \
M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \
\
/** Obsolete settings. Kept for backward compatibility only. */ \
M(SettingUInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \
M(SettingUInt64, check_delay_period, 60, "Obsolete setting, does nothing.", 0) \
DECLARE_SETTINGS_COLLECTION(LIST_OF_MERGE_TREE_SETTINGS)
M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \
M(UInt64, check_delay_period, 60, "Obsolete setting, does nothing.", 0) \
/// Settings that should not change after the creation of a table.
#define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \
M(index_granularity)
DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS)
/** Settings for the MergeTree family of engines.
* Could be loaded from config or from a CREATE TABLE query (SETTINGS clause).
*/
struct MergeTreeSettings : public BaseSettings<MergeTreeSettingsTraits>
{
void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
/// NOTE: will rewrite the AST to add immutable settings.

View File

@ -3,7 +3,6 @@
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/Exception.h>
#include <Core/SettingsCollectionImpl.h>
namespace DB
{
@ -14,7 +13,7 @@ namespace ErrorCodes
extern const int UNKNOWN_SETTING;
}
IMPLEMENT_SETTINGS_COLLECTION(RabbitMQSettings, LIST_OF_RABBITMQ_SETTINGS)
IMPLEMENT_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
void RabbitMQSettings::loadFromQuery(ASTStorage & storage_def)
{

View File

@ -1,27 +1,29 @@
#pragma once
#include <Core/SettingsCollection.h>
#include <Core/BaseSettings.h>
namespace DB
{
class ASTStorage;
struct RabbitMQSettings : public SettingsCollection<RabbitMQSettings>
{
#define LIST_OF_RABBITMQ_SETTINGS(M) \
M(SettingString, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \
M(SettingString, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \
M(SettingString, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \
M(SettingString, rabbitmq_format, "", "The message format.", 0) \
M(SettingChar, rabbitmq_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \
M(SettingString, rabbitmq_exchange_type, "default", "The exchange type.", 0) \
M(SettingUInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \
M(SettingUInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \
M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \
#define LIST_OF_RABBITMQ_SETTINGS(M) \
M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \
M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \
M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \
M(String, rabbitmq_format, "", "The message format.", 0) \
M(Char, rabbitmq_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \
M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \
M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \
M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \
M(Bool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \
DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS)
DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS)
struct RabbitMQSettings : public BaseSettings<RabbitMQSettingsTraits>
{
void loadFromQuery(ASTStorage & storage_def);
};
};
}

View File

@ -124,17 +124,17 @@ void registerStorageJoin(StorageFactory & factory)
for (const auto & setting : args.storage_def->settings->changes)
{
if (setting.name == "join_use_nulls")
join_use_nulls.set(setting.value);
join_use_nulls = setting.value;
else if (setting.name == "max_rows_in_join")
max_rows_in_join.set(setting.value);
max_rows_in_join = setting.value;
else if (setting.name == "max_bytes_in_join")
max_bytes_in_join.set(setting.value);
max_bytes_in_join = setting.value;
else if (setting.name == "join_overflow_mode")
join_overflow_mode.set(setting.value);
join_overflow_mode = setting.value;
else if (setting.name == "join_any_take_last_row")
join_any_take_last_row.set(setting.value);
join_any_take_last_row = setting.value;
else if (setting.name == "any_join_distinct_right_table_keys")
old_any_join.set(setting.value);
old_any_join = setting.value;
else
throw Exception(
"Unknown setting " + setting.name + " for storage " + args.engine_name,

View File

@ -236,7 +236,7 @@ private:
size_t total_tables;
std::shared_ptr<const ContextAccess> access;
String query_id;
SettingSeconds lock_acquire_timeout;
std::chrono::milliseconds lock_acquire_timeout;
};

View File

@ -20,13 +20,13 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes()
void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
for (const auto & setting : context.getMergeTreeSettings())
for (auto setting : context.getMergeTreeSettings().all())
{
res_columns[0]->insert(setting.getName().toString());
res_columns[1]->insert(setting.getValueAsString());
res_columns[2]->insert(setting.isChanged());
res_columns[3]->insert(setting.getDescription().toString());
res_columns[4]->insert(setting.getType().toString());
res_columns[0]->insert(setting.getName());
res_columns[1]->insert(setting.getValueString());
res_columns[2]->insert(setting.isValueChanged());
res_columns[3]->insert(setting.getDescription());
res_columns[4]->insert(setting.getTypeName());
}
}

View File

@ -30,13 +30,13 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context
{
const Settings & settings = context.getSettingsRef();
auto settings_constraints = context.getSettingsConstraints();
for (const auto & setting : settings)
for (auto setting : settings.all())
{
StringRef setting_name = setting.getName();
res_columns[0]->insert(setting_name.toString());
res_columns[1]->insert(setting.getValueAsString());
res_columns[2]->insert(setting.isChanged());
res_columns[3]->insert(setting.getDescription().toString());
const auto & setting_name = setting.getName();
res_columns[0]->insert(setting_name);
res_columns[1]->insert(setting.getValueString());
res_columns[2]->insert(setting.isValueChanged());
res_columns[3]->insert(setting.getDescription());
Field min, max;
bool read_only = false;
@ -45,9 +45,9 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context
/// These two columns can accept strings only.
if (!min.isNull())
min = Settings::valueToString(setting_name, min);
min = Settings::valueToStringUtil(setting_name, min);
if (!max.isNull())
max = Settings::valueToString(setting_name, max);
max = Settings::valueToStringUtil(setting_name, max);
if (!read_only)
{
@ -60,7 +60,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context
res_columns[4]->insert(min);
res_columns[5]->insert(max);
res_columns[6]->insert(read_only);
res_columns[7]->insert(setting.getType().toString());
res_columns[7]->insert(setting.getTypeName());
}
}

View File

@ -126,12 +126,12 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns
}
}
if ((element.setting_index != static_cast<size_t>(-1))
if (!element.setting_name.empty()
&& (!element.value.isNull() || !element.min_value.isNull() || !element.max_value.isNull() || element.readonly))
{
auto setting_name = Settings::getName(element.setting_index);
const auto & setting_name = element.setting_name;
column_index.push_back(index++);
column_setting_name.insertData(setting_name.data, setting_name.size);
column_setting_name.insertData(setting_name.data(), setting_name.size());
column_setting_name_null_map.push_back(false);
if (element.value.isNull())
@ -141,7 +141,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns
}
else
{
String str = Settings::valueToString(element.setting_index, element.value);
String str = Settings::valueToStringUtil(setting_name, element.value);
column_value.insertData(str.data(), str.length());
column_value_null_map.push_back(false);
}
@ -153,7 +153,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns
}
else
{
String str = Settings::valueToString(element.setting_index, element.min_value);
String str = Settings::valueToStringUtil(setting_name, element.min_value);
column_min.insertData(str.data(), str.length());
column_min_null_map.push_back(false);
}
@ -165,7 +165,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns
}
else
{
String str = Settings::valueToString(element.setting_index, element.max_value);
String str = Settings::valueToStringUtil(setting_name, element.max_value);
column_max.insertData(str.data(), str.length());
column_max_null_map.push_back(false);
}

View File

@ -0,0 +1,4 @@
<yandex>
<!-- Comma-separated list of prefixes for user-defined settings. -->
<custom_settings_prefixes>custom_</custom_settings_prefixes>
</yandex>

View File

@ -1,4 +1,4 @@
send_timeout 300 0 \N \N 0 SettingSeconds
replicated_max_parallel_sends 0 0 Limit parallel sends. SettingUInt64
send_timeout 300 0 \N \N 0 Seconds
replicated_max_parallel_sends 0 0 Limit parallel sends. UInt64
1
1

View File

@ -2,33 +2,33 @@ select * from system.settings where name = 'send_timeout';
select * from system.merge_tree_settings order by length(description) limit 1;
with [
'SettingSeconds',
'SettingBool',
'SettingInt64',
'SettingString',
'SettingChar',
'SettingLogsLevel',
'SettingURI',
'SettingFloat',
'SettingUInt64',
'SettingMaxThreads',
'SettingMilliseconds',
'SettingJoinStrictness',
'SettingJoinAlgorithm',
'SettingOverflowMode',
'SettingTotalsMode',
'SettingLoadBalancing',
'SettingOverflowModeGroupBy',
'SettingDateTimeInputFormat',
'SettingDistributedProductMode'
'Seconds',
'Bool',
'Int64',
'String',
'Char',
'LogsLevel',
'URI',
'Float',
'UInt64',
'MaxThreads',
'Milliseconds',
'JoinStrictness',
'JoinAlgorithm',
'OverflowMode',
'TotalsMode',
'LoadBalancing',
'OverflowModeGroupBy',
'DateTimeInputFormat',
'DistributedProductMode'
] as types select hasAll(arrayDistinct(groupArray(type)), types) from system.settings;
with [
'SettingSeconds',
'SettingBool',
'SettingInt64',
'SettingString',
'SettingFloat',
'SettingUInt64',
'SettingMaxThreads'
'Seconds',
'Bool',
'Int64',
'String',
'Float',
'UInt64',
'MaxThreads'
] as types select hasAll(arrayDistinct(groupArray(type)), types) from system.merge_tree_settings;

View File

@ -0,0 +1,17 @@
5 UInt8
-177 Int16
98.11 Float64
abc def String
custom_a UInt64_5
custom_b Int64_-177
custom_c Float64_98.11
custom_d \'abc def\'
changed String
\N Nullable(Nothing)
50000 UInt16
1.11 Float64
custom_a \'changed\'
custom_b NULL
custom_c UInt64_50000
custom_d Float64_1.11
0 UInt8

View File

@ -0,0 +1,25 @@
SET custom_a = 5;
SET custom_b = -177;
SET custom_c = 98.11;
SET custom_d = 'abc def';
SELECT getSetting('custom_a') as v, toTypeName(v);
SELECT getSetting('custom_b') as v, toTypeName(v);
SELECT getSetting('custom_c') as v, toTypeName(v);
SELECT getSetting('custom_d') as v, toTypeName(v);
SELECT name, value FROM system.settings WHERE name LIKE 'custom_%' ORDER BY name;
SET custom_a = 'changed';
SET custom_b = NULL;
SET custom_c = 50000;
SET custom_d = 1.11;
SELECT getSetting('custom_a') as v, toTypeName(v);
SELECT getSetting('custom_b') as v, toTypeName(v);
SELECT getSetting('custom_c') as v, toTypeName(v);
SELECT getSetting('custom_d') as v, toTypeName(v);
SELECT name, value FROM system.settings WHERE name LIKE 'custom_%' ORDER BY name;
SELECT getSetting('custom_e') as v, toTypeName(v); -- { serverError 115 } -- Setting not found.
SET custom_e = 0;
SELECT getSetting('custom_e') as v, toTypeName(v);
SET invalid_custom = 8; -- { serverError 115 } -- Setting is neither a builtin nor started with one of the registered prefixes for user-defined settings.