Revert changes with complex SettingsChanges

This commit is contained in:
kssenii 2023-02-03 21:03:13 +01:00
parent 12b62de374
commit 5f124263a6
40 changed files with 115 additions and 104 deletions

View File

@ -548,7 +548,7 @@ void Client::printChangedSettings() const
{
fmt::print(stderr, ", ");
}
fmt::print(stderr, "{} = '{}'", changes[i].getName(), changes[i].getValueString());
fmt::print(stderr, "{} = '{}'", changes[i].name, toString(changes[i].value));
}
fmt::print(stderr, "\n");
}

View File

@ -175,19 +175,19 @@ template <class T>
bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure)
{
Field current_value;
bool has_current_value = current_settings.tryGet(change.getName(), current_value);
bool has_current_value = current_settings.tryGet(change.name, current_value);
/// Setting isn't checked if value has not changed.
if (has_current_value && change.getFieldValue() == current_value)
if (has_current_value && change.value == current_value)
return false;
if (throw_on_failure)
new_value = T::castValueUtil(change.getName(), change.getFieldValue());
new_value = T::castValueUtil(change.name, change.value);
else
{
try
{
new_value = T::castValueUtil(change.getName(), change.getFieldValue());
new_value = T::castValueUtil(change.name, change.value);
}
catch (...)
{
@ -204,7 +204,7 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel
bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const
{
const String & setting_name = change.getName();
const String & setting_name = change.name;
if (setting_name == "profile")
return true;
@ -219,7 +219,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh
{
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
{
if (const auto hints = current_settings.getHints(change.getName()); !hints.empty())
if (const auto hints = current_settings.getHints(change.name); !hints.empty())
{
e.addMessage(fmt::format("Maybe you meant {}", toString(hints)));
}
@ -242,12 +242,12 @@ bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings,
Field new_value;
if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION))
return false;
return getMergeTreeChecker(change.getName()).check(change, new_value, reaction);
return getMergeTreeChecker(change.name).check(change, new_value, reaction);
}
bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const
{
const String & setting_name = change.getName();
const String & setting_name = change.name;
auto less_or_cannot_compare = [=](const Field & left, const Field & right)
{
@ -298,7 +298,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
setting_name, applyVisitor(FieldVisitorToString(), min_value));
}
else
change.setValue(min_value);
change.value = min_value;
}
if (!max_value.isNull() && less_or_cannot_compare(max_value, new_value))
@ -309,7 +309,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
setting_name, applyVisitor(FieldVisitorToString(), max_value));
}
else
change.setValue(max_value);
change.value = max_value;
}
return true;

View File

@ -12,7 +12,7 @@ namespace Poco::Util
namespace DB
{
struct Settings;
class SettingChange;
struct SettingChange;
struct MergeTreeSettings;
class SettingsChanges;
class AccessControl;

View File

@ -186,7 +186,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s
{
buffer << '\n' << std::string(indent + 2, ' ') << "SETTINGS";
for (const auto & change : settings_changes)
buffer << fmt::format(" {}={}", change.getName(), toString(change.getFieldValue()));
buffer << fmt::format(" {}={}", change.name, toString(change.value));
}
}

View File

@ -41,16 +41,16 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query)
const auto & settings = query.settings->as<const ASTSetQuery &>().changes;
for (const auto & setting : settings)
{
if (setting.getName() == "compression_level")
res.compression_level = static_cast<int>(SettingFieldInt64{setting.getFieldValue()}.value);
if (setting.name == "compression_level")
res.compression_level = static_cast<int>(SettingFieldInt64{setting.value}.value);
else
#define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \
if (setting.getName() == #NAME) \
res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \
if (setting.name == #NAME) \
res.NAME = SettingField##TYPE{setting.value}.value; \
else
LIST_OF_BACKUP_SETTINGS(GET_SETTINGS_FROM_BACKUP_QUERY_HELPER)
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName());
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
}
}

View File

@ -177,11 +177,11 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
for (const auto & setting : settings)
{
#define GET_SETTINGS_FROM_RESTORE_QUERY_HELPER(TYPE, NAME) \
if (setting.getName() == #NAME) \
res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \
if (setting.name == #NAME) \
res.NAME = SettingField##TYPE{setting.value}.value; \
else
LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER)
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName());
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
}
}

View File

@ -1638,8 +1638,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
/// Save all changes in settings to avoid losing them if the connection is lost.
for (const auto & change : set_query->changes)
{
if (change.getName() == "profile")
current_profile = change.getFieldValue().safeGet<String>();
if (change.name == "profile")
current_profile = change.value.safeGet<String>();
else
global_context->applySettingChange(change);
}

View File

@ -151,7 +151,7 @@ ConfigurationPtr createConfiguration(const std::string & root_name, const Settin
auto config = Configuration::createEmptyConfiguration(root_name);
for (const auto & setting : settings)
Configuration::setConfigValue<String>(*config, setting.getName(), convertFieldToString(setting.getFieldValue()));
Configuration::setConfigValue<String>(*config, setting.name, convertFieldToString(setting.value));
return config;
}

View File

@ -182,18 +182,18 @@ public:
std::unordered_map<std::string, Field> result_changes_map;
for (const auto & change : query.changes)
{
auto [it, inserted] = result_changes_map.emplace(change.getName(), change.getFieldValue());
auto [it, inserted] = result_changes_map.emplace(change.name, change.value);
if (!inserted)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Value with key `{}` is used twice in the SET query (collection name: {})",
change.getName(), query.collection_name);
change.name, query.collection_name);
}
}
for (const auto & change : create_query.changes)
result_changes_map.emplace(change.getName(), change.getFieldValue());
result_changes_map.emplace(change.name, change.value);
for (const auto & delete_key : query.delete_keys)
{
@ -254,7 +254,7 @@ private:
std::set<std::string, std::less<>> keys;
for (const auto & change : query.changes)
keys.insert(change.getName());
keys.insert(change.name);
return NamedCollection::create(
*config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true);
@ -379,7 +379,7 @@ void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr contex
auto collection_lock = collection->lock();
for (const auto & change : query.changes)
collection->setOrUpdate<String, true>(change.getName(), convertFieldToString(change.getFieldValue()));
collection->setOrUpdate<String, true>(change.name, convertFieldToString(change.value));
for (const auto & key : query.delete_keys)
collection->remove<true>(key);

View File

@ -293,7 +293,7 @@ SettingsChanges BaseSettings<TTraits>::changes() const
template <typename TTraits>
void BaseSettings<TTraits>::applyChange(const SettingChange & change)
{
set(change.getName(), change.getFieldValue());
set(change.name, change.value);
}
template <typename TTraits>

View File

@ -755,9 +755,9 @@ void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_cha
for (const auto & change : settings_changes)
{
auto it = std::find_if(storage_settings.begin(), storage_settings.end(),
[&](const auto & prev){ return prev.getName() == change.getName(); });
[&](const auto & prev){ return prev.name == change.name; });
if (it != storage_settings.end())
it->setValue(change.getFieldValue());
it->value = change.value;
else
storage_settings.push_back(change);
}

View File

@ -42,7 +42,7 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def)
SettingsChanges & changes = storage_def.settings->changes;
#define ADD_IF_ABSENT(NAME) \
if (std::find_if(changes.begin(), changes.end(), \
[](const SettingChange & c) { return c.getName() == #NAME; }) \
[](const SettingChange & c) { return c.name == #NAME; }) \
== changes.end()) \
changes.push_back(SettingChange{#NAME, static_cast<Field>(NAME)});

View File

@ -139,18 +139,18 @@ void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges
for (const auto & change : settings_changes)
{
if (!settings->has(change.getName()))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.getName());
if (!settings->has(change.name))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.name);
if ((change.getName() == "materialized_postgresql_tables_list"))
if ((change.name == "materialized_postgresql_tables_list"))
{
if (!query_context->isInternalQuery())
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.getName());
throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.name);
need_update_on_disk = true;
}
else if ((change.getName() == "materialized_postgresql_allow_automatic_update")
|| (change.getName() == "materialized_postgresql_max_block_size"))
else if ((change.name == "materialized_postgresql_allow_automatic_update")
|| (change.name == "materialized_postgresql_max_block_size"))
{
replication_handler->setSetting(change);
need_update_on_disk = true;

View File

@ -127,9 +127,9 @@ void buildLayoutConfiguration(
root->appendChild(settings_element);
for (const auto & change : settings->changes)
{
AutoPtr<Element> setting_change_element(doc->createElement(change.getName()));
AutoPtr<Element> setting_change_element(doc->createElement(change.name));
settings_element->appendChild(setting_change_element);
AutoPtr<Text> setting_value(doc->createTextNode(convertFieldToString(change.getFieldValue())));
AutoPtr<Text> setting_value(doc->createTextNode(convertFieldToString(change.value)));
setting_change_element->appendChild(setting_value);
}
}
@ -531,9 +531,9 @@ void buildSourceConfiguration(
outer_element->appendChild(settings_element);
for (const auto & change : settings->changes)
{
AutoPtr<Element> setting_change_element(doc->createElement(change.getName()));
AutoPtr<Element> setting_change_element(doc->createElement(change.name));
settings_element->appendChild(setting_change_element);
AutoPtr<Text> setting_value(doc->createTextNode(convertFieldToString(change.getFieldValue())));
AutoPtr<Text> setting_value(doc->createTextNode(convertFieldToString(change.value)));
setting_change_element->appendChild(setting_value);
}
}

View File

@ -79,7 +79,7 @@ public:
auto is_query_cache_related_setting = [](const auto & change)
{
const auto & name = change.getName();
const auto & name = change.name;
return name == "allow_experimental_query_cache"
|| name.starts_with("query_cache")
|| name.ends_with("query_cache");

View File

@ -1513,11 +1513,11 @@ void Context::applySettingChange(const SettingChange & change)
{
try
{
setSetting(change.getName(), change.getFieldValue());
setSetting(change.name, change.value);
}
catch (Exception & e)
{
e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.getName(), change.getValueString()));
e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.name, toString(change.value)));
throw;
}
}

View File

@ -313,28 +313,28 @@ ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
for (const auto & change : set_query.changes)
{
if (!settings.has(change.getName()))
if (!settings.has(change.name))
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting \"{}\" for EXPLAIN {} query. "
"Supported settings: {}", change.getName(), Settings::name, settings.getSettingsList());
"Supported settings: {}", change.name, Settings::name, settings.getSettingsList());
if (change.getFieldValue().getType() != Field::Types::UInt64)
if (change.value.getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::INVALID_SETTING_VALUE,
"Invalid type {} for setting \"{}\" only integer settings are supported",
change.getFieldValue().getTypeName(), change.getName());
change.value.getTypeName(), change.name);
if (settings.hasBooleanSetting(change.getName()))
if (settings.hasBooleanSetting(change.name))
{
auto value = change.getFieldValue().get<UInt64>();
auto value = change.value.get<UInt64>();
if (value > 1)
throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid value {} for setting \"{}\". "
"Expected boolean type", value, change.getName());
"Expected boolean type", value, change.name);
settings.setBooleanSetting(change.getName(), value);
settings.setBooleanSetting(change.name, value);
}
else
{
auto value = change.getFieldValue().get<UInt64>();
settings.setIntegerSetting(change.getName(), value);
auto value = change.value.get<UInt64>();
settings.setIntegerSetting(change.name, value);
}
}

View File

@ -28,9 +28,9 @@ void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & setti
else
first = false;
formatSettingName(change.getName(), settings.ostr);
formatSettingName(change.name, settings.ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue());
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
}

View File

@ -184,7 +184,7 @@ namespace
changes,
[](const SettingChange & change)
{
const String & name = change.getName();
const String & name = change.name;
return (name == "internal") || (name == "async") || (name == "host_id");
});

View File

@ -31,10 +31,10 @@ void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & sett
else
first = false;
formatSettingName(change.getName(), settings.ostr);
formatSettingName(change.name, settings.ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue());
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
}

View File

@ -118,7 +118,7 @@ void ASTDictionarySettings::formatImpl(const FormatSettings & settings,
if (it != changes.begin())
settings.ostr << ", ";
settings.ostr << it->getName() << " = " << applyVisitor(FieldVisitorToString(), it->getFieldValue());
settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value);
}
settings.ostr << (settings.hilite ? hilite_none : "") << ")";
}

View File

@ -49,7 +49,7 @@ void InsertQuerySettingsPushDownMatcher::visit(ASTSelectQuery & select_query, AS
{
auto it = std::find_if(insert_settings.begin(), insert_settings.end(), [&](auto & select_setting)
{
return select_setting.getName() == setting.getName();
return select_setting.name == setting.name;
});
if (it == insert_settings.end())
insert_settings.push_back(setting);

View File

@ -7,7 +7,7 @@ namespace DB
{
class ASTSelectQuery;
class SettingChange;
struct SettingChange;
class SettingsChanges;
/// Pushdown SETTINGS clause to the INSERT from the SELECT query:

View File

@ -304,7 +304,7 @@ namespace
changes = assert_cast<ASTSetQuery *>(settings.get())->changes;
}
boost::remove_erase_if(changes, [](const SettingChange & change) { return change.getName() == "async"; });
boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; });
changes.emplace_back("async", async);
auto new_settings = std::make_shared<ASTSetQuery>();

View File

@ -44,12 +44,12 @@ void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query
{
auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting)
{
return select_setting.getName() == setting.getName();
return select_setting.name == setting.name;
});
if (it == select_settings.end())
select_settings.push_back(setting);
else
it->setValue(setting.getFieldValue());
it->value = setting.value;
}
}

View File

@ -7,7 +7,7 @@ namespace DB
{
class ASTSelectQuery;
class SettingChange;
struct SettingChange;
class SettingsChanges;
/// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query:

View File

@ -619,11 +619,11 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
auto & settings_from_storage = metadata.settings_changes->as<ASTSetQuery &>().changes;
for (const auto & change : settings_changes)
{
auto finder = [&change](const SettingChange & c) { return c.getName() == change.getName(); };
auto finder = [&change](const SettingChange & c) { return c.name == change.name; };
auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder);
if (it != settings_from_storage.end())
it->setValue(change.getFieldValue());
it->value = change.value;
else
settings_from_storage.push_back(change);
}
@ -633,7 +633,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
auto & settings_from_storage = metadata.settings_changes->as<ASTSetQuery &>().changes;
for (const auto & setting_name : settings_resets)
{
auto finder = [&setting_name](const SettingChange & c) { return c.getName() == setting_name; };
auto finder = [&setting_name](const SettingChange & c) { return c.name == setting_name; };
auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder);
if (it != settings_from_storage.end())

View File

@ -3049,8 +3049,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
for (const auto & changed_setting : new_changes)
{
const auto & setting_name = changed_setting.getName();
const auto & new_value = changed_setting.getFieldValue();
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);
@ -3076,7 +3076,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
/// Check if it is safe to reset the settings
for (const auto & current_setting : current_changes)
{
const auto & setting_name = current_setting.getName();
const auto & setting_name = current_setting.name;
const Field * new_value = new_changes.tryGet(setting_name);
/// Prevent unsetting readonly setting
if (MergeTreeSettings::isReadonlySetting(setting_name) && !new_value)
@ -3174,9 +3174,9 @@ void MergeTreeData::changeSettings(
for (const auto & change : new_changes)
{
if (change.getName() == "storage_policy")
if (change.name == "storage_policy")
{
StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.getFieldValue().safeGet<String>());
StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.value.safeGet<String>());
StoragePolicyPtr old_storage_policy = getStoragePolicy();
/// StoragePolicy of different version or name is guaranteed to have different pointer

View File

@ -181,6 +181,14 @@ namespace
hash.update(x.data.size());
hash.update(x.data.data(), x.data.size());
}
void operator() (const CustomType & x) const
{
UInt8 type = Field::Types::CustomType;
hash.update(type);
auto result = x.toString();
hash.update(result.size());
hash.update(result.data(), result.size());
}
void operator() (const bool & x) const
{
UInt8 type = Field::Types::Bool;

View File

@ -65,7 +65,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def)
#define ADD_IF_ABSENT(NAME) \
if (std::find_if(changes.begin(), changes.end(), \
[](const SettingChange & c) { return c.getName() == #NAME; }) \
[](const SettingChange & c) { return c.name == #NAME; }) \
== changes.end()) \
changes.push_back(SettingChange{#NAME, (NAME).value});

View File

@ -17,7 +17,7 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/SettingValueFromAST.h>
#include <Parsers/FieldFromAST.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
@ -592,16 +592,19 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.storage_def->settings)
{
for (auto & change : args.storage_def->settings->changes)
for (auto & [name, value] : args.storage_def->settings->changes)
{
auto value = change.getValue();
auto * ast_value = dynamic_cast<SettingValueFromAST *>(value.get());
if (ast_value && isDiskFunction(ast_value->value))
CustomType custom;
if (value.tryGet<CustomType>(custom) && 0 == strcmp(custom.getTypeName(), "AST"))
{
const auto & ast_function = assert_cast<const ASTFunction &>(*ast_value->value);
auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context);
ast_value->setField(disk_name);
break;
auto ast = dynamic_cast<const FieldFromASTImpl &>(custom.getImpl()).ast;
if (ast && isDiskFunction(ast))
{
const auto & ast_function = assert_cast<const ASTFunction &>(*ast);
auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context);
value = disk_name;
break;
}
}
}
}

View File

@ -731,10 +731,10 @@ void MaterializedPostgreSQLConsumer::removeNested(const String & postgres_table_
void MaterializedPostgreSQLConsumer::setSetting(const SettingChange & setting)
{
if (setting.getName() == "materialized_postgresql_max_block_size")
max_block_size = setting.getFieldValue().safeGet<UInt64>();
else if (setting.getName() == "materialized_postgresql_allow_automatic_update")
allow_automatic_update = setting.getFieldValue().safeGet<bool>();
if (setting.name == "materialized_postgresql_max_block_size")
max_block_size = setting.value.safeGet<UInt64>();
else if (setting.name == "materialized_postgresql_allow_automatic_update")
allow_automatic_update = setting.value.safeGet<bool>();
}

View File

@ -13,7 +13,7 @@
namespace DB
{
class SettingChange;
struct SettingChange;
struct StorageInfo
{

View File

@ -11,7 +11,7 @@ namespace DB
{
class StorageMaterializedPostgreSQL;
class SettingChange;
struct SettingChange;
class PostgreSQLReplicationHandler : WithContext
{

View File

@ -1218,9 +1218,9 @@ void registerStorageFile(StorageFactory & factory)
const auto & changes = factory_args.getContext()->getSettingsRef().changes();
for (const auto & change : changes)
{
if (user_format_settings.has(change.getName()))
if (user_format_settings.has(change.name))
{
user_format_settings.set(change.getName(), change.getFieldValue());
user_format_settings.set(change.name, change.value);
}
}

View File

@ -305,8 +305,8 @@ void registerStorageJoin(StorageFactory & factory)
{
for (const auto & setting : args.storage_def->settings->changes)
{
const auto & setting_name = setting.getName();
const auto & setting_value = setting.getFieldValue();
const auto & setting_name = setting.name;
const auto & setting_value = setting.value;
if (setting_name == "join_use_nulls")
join_use_nulls = setting_value;
else if (setting_name == "max_rows_in_join")

View File

@ -10,8 +10,8 @@ String getDiskName(ASTStorage & storage_def)
{
SettingsChanges changes = storage_def.settings->changes;
for (const auto & change : changes)
if (change.getName() == "disk")
return change.getFieldValue().safeGet<String>();
if (change.name == "disk")
return change.value.safeGet<String>();
}
return "default";
}

View File

@ -1451,8 +1451,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
const auto & changes = args.getContext()->getSettingsRef().changes();
for (const auto & change : changes)
{
if (user_format_settings.has(change.getName()))
user_format_settings.set(change.getName(), change.getFieldValue());
if (user_format_settings.has(change.name))
user_format_settings.set(change.name, change.value);
}
// Apply changes from SETTINGS clause, with validation.

View File

@ -983,9 +983,9 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum
const auto & changes = args.getContext()->getSettingsRef().changes();
for (const auto & change : changes)
{
if (user_format_settings.has(change.getName()))
if (user_format_settings.has(change.name))
{
user_format_settings.set(change.getName(), change.getFieldValue());
user_format_settings.set(change.name, change.value);
}
}

View File

@ -122,8 +122,8 @@ static void fillCommonColumns(MutableColumns & res_columns, size_t & col, const
for (const auto & change : *task.entry.settings)
{
Tuple pair;
pair.push_back(change.getName());
pair.push_back(toString(change.getFieldValue()));
pair.push_back(change.name);
pair.push_back(toString(change.value));
settings_map.push_back(std::move(pair));
}
}