Merge pull request #43903 from ClickHouse/merge_tree_settings_constraints

Constraints on merge tree settings
This commit is contained in:
Sergei Trifonov 2022-12-09 14:16:14 +01:00 committed by GitHub
commit 7c37e48092
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 342 additions and 72 deletions

View File

@ -91,4 +91,21 @@ Code: 452, e.displayText() = DB::Exception: Setting force_index_by_date should n
**Note:** the `default` profile has special handling: all the constraints defined for the `default` profile become the default constraints, so they restrict all the users until theyre overridden explicitly for these users.
## Constraints on Merge Tree Settings
It is possible to set constraints for [merge tree settings](merge-tree-settings.md). There constraints are applied when table with merge tree engine is created or its storage settings are altered. Name of merge tree setting must be prepended by `merge_tree_` prefix when referenced in `<constraint>` section.
**Example:** Forbid to create new tables with explicitly specified `storage_policy`
``` xml
<profiles>
<default>
<constraints>
<merge_tree_storage_policy>
<const/>
</merge_tree_storage_policy>
</constraints>
</default>
</profiles>
```
[Original article](https://clickhouse.com/docs/en/operations/settings/constraints_on_settings/) <!--hide-->

View File

@ -16,9 +16,11 @@
#include <Access/ExternalAuthenticators.h>
#include <Access/AccessChangesNotifier.h>
#include <Access/AccessBackup.h>
#include <Access/resolveSetting.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Core/Settings.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <base/defines.h>
#include <IO/Operators.h>
#include <Poco/AccessExpireCache.h>
@ -38,7 +40,6 @@ namespace ErrorCodes
extern const int AUTHENTICATION_FAILED;
}
namespace
{
void checkForUsersNotInMainConfig(
@ -103,7 +104,7 @@ public:
bool isSettingNameAllowed(std::string_view setting_name) const
{
if (Settings::hasBuiltin(setting_name))
if (settingIsBuiltin(setting_name))
return true;
std::lock_guard lock{mutex};

View File

@ -174,7 +174,6 @@ private:
void initialize();
void setUser(const UserPtr & user_) const;
void setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & roles_info_) const;
void setSettingsAndConstraints() const;
void calculateAccessRights() const;
template <bool throw_if_denied, bool grant_option>

View File

@ -1,13 +1,15 @@
#include <string_view>
#include <Access/SettingsConstraints.h>
#include <Access/resolveSetting.h>
#include <Access/AccessControl.h>
#include <Core/Settings.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Common/FieldVisitorToString.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
@ -18,7 +20,6 @@ namespace ErrorCodes
extern const int UNKNOWN_SETTING;
}
SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_)
{
}
@ -35,19 +36,28 @@ void SettingsConstraints::clear()
constraints.clear();
}
void SettingsConstraints::set(const String & setting_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability)
void SettingsConstraints::set(const String & full_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability)
{
auto & constraint = constraints[setting_name];
auto & constraint = constraints[full_name];
if (!min_value.isNull())
constraint.min_value = Settings::castValueUtil(setting_name, min_value);
constraint.min_value = settingCastValueUtil(full_name, min_value);
if (!max_value.isNull())
constraint.max_value = Settings::castValueUtil(setting_name, max_value);
constraint.max_value = settingCastValueUtil(full_name, max_value);
constraint.writability = writability;
}
void SettingsConstraints::get(const Settings & current_settings, std::string_view setting_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const
void SettingsConstraints::get(const Settings & current_settings, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const
{
auto checker = getChecker(current_settings, setting_name);
// NOTE: for `Settings` short name is equal to full name
auto checker = getChecker(current_settings, short_name);
min_value = checker.constraint.min_value;
max_value = checker.constraint.max_value;
writability = checker.constraint.writability;
}
void SettingsConstraints::get(const MergeTreeSettings &, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const
{
auto checker = getMergeTreeChecker(short_name);
min_value = checker.constraint.min_value;
max_value = checker.constraint.max_value;
writability = checker.constraint.writability;
@ -97,6 +107,17 @@ void SettingsConstraints::check(const Settings & current_settings, SettingsChang
});
}
void SettingsConstraints::check(const MergeTreeSettings & current_settings, const SettingChange & change) const
{
checkImpl(current_settings, const_cast<SettingChange &>(change), THROW_ON_VIOLATION);
}
void SettingsConstraints::check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const
{
for (const auto & change : changes)
check(current_settings, change);
}
void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const
{
boost::range::remove_erase_if(
@ -107,6 +128,36 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang
});
}
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.name, current_value);
/// Setting isn't checked if value has not changed.
if (has_current_value && change.value == current_value)
return false;
if (throw_on_failure)
new_value = T::castValueUtil(change.name, change.value);
else
{
try
{
new_value = T::castValueUtil(change.name, change.value);
}
catch (...)
{
return false;
}
}
/// Setting isn't checked if value has not changed.
if (has_current_value && new_value == current_value)
return false;
return true;
}
bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const
{
@ -115,26 +166,6 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh
if (setting_name == "profile")
return true;
bool cannot_cast;
auto cast_value = [&](const Field & x) -> Field
{
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 {};
}
}
};
if (reaction == THROW_ON_VIOLATION)
{
try
@ -156,27 +187,21 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh
else if (!access_control->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;
}
Field new_value;
if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION))
return false;
return getChecker(current_settings, setting_name).check(change, new_value, reaction);
}
bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const
{
Field new_value;
if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION))
return false;
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.name;
@ -185,16 +210,13 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
{
if (reaction == THROW_ON_VIOLATION)
return applyVisitor(FieldVisitorAccurateLess{}, left, right);
else
try
{
try
{
return applyVisitor(FieldVisitorAccurateLess{}, left, right);
}
catch (...)
{
return true;
}
return applyVisitor(FieldVisitorAccurateLess{}, left, right);
}
catch (...)
{
return true;
}
};
@ -280,6 +302,14 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu
return Checker(it->second);
}
SettingsConstraints::Checker SettingsConstraints::getMergeTreeChecker(std::string_view short_name) const
{
auto it = constraints.find(settingFullName<MergeTreeSettings>(short_name));
if (it == constraints.end())
return Checker(); // Allowed
return Checker(it->second);
}
bool SettingsConstraints::Constraint::operator==(const Constraint & other) const
{
return writability == other.writability && min_value == other.min_value && max_value == other.max_value;

View File

@ -12,6 +12,7 @@ namespace Poco::Util
namespace DB
{
struct Settings;
struct MergeTreeSettings;
struct SettingChange;
class SettingsChanges;
class AccessControl;
@ -65,8 +66,9 @@ public:
void clear();
bool empty() const { return constraints.empty(); }
void set(const String & setting_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability);
void get(const Settings & current_settings, std::string_view setting_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const;
void set(const String & full_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability);
void get(const Settings & current_settings, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const;
void get(const MergeTreeSettings & current_settings, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const;
void merge(const SettingsConstraints & other);
@ -75,6 +77,10 @@ public:
void check(const Settings & current_settings, const SettingsChanges & changes) const;
void check(const Settings & current_settings, SettingsChanges & changes) const;
/// Checks whether `change` violates these constraints and throws an exception if so. (setting short name is expected inside `changes`)
void check(const MergeTreeSettings & current_settings, const SettingChange & change) const;
void check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const;
/// Checks whether `change` violates these and clamps the `change` if so.
void clamp(const Settings & current_settings, SettingsChanges & changes) const;
@ -137,8 +143,10 @@ private:
};
bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const;
bool checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const;
Checker getChecker(const Settings & current_settings, std::string_view setting_name) const;
Checker getMergeTreeChecker(std::string_view short_name) const;
// Special container for heterogeneous lookups: to avoid `String` construction during `find(std::string_view)`
using Constraints = std::unordered_map<String, Constraint, StringHash, std::equal_to<>>;

View File

@ -4,6 +4,7 @@
#include <Access/User.h>
#include <Access/SettingsProfile.h>
#include <Access/AccessControl.h>
#include <Access/resolveSetting.h>
#include <Access/AccessChangesNotifier.h>
#include <Dictionaries/IDictionary.h>
#include <Common/Config/ConfigReloader.h>
@ -451,9 +452,9 @@ namespace
for (const String & constraint_type : constraint_types)
{
if (constraint_type == "min")
profile_element.min_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type));
profile_element.min_value = settingStringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type));
else if (constraint_type == "max")
profile_element.max_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type));
profile_element.max_value = settingStringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type));
else if (constraint_type == "readonly" || constraint_type == "const")
{
writability_count++;
@ -517,7 +518,7 @@ namespace
SettingsProfileElement profile_element;
profile_element.setting_name = setting_name;
profile_element.value = Settings::stringToValueUtil(setting_name, config.getString(profile_config + "." + key));
profile_element.value = settingStringToValueUtil(setting_name, config.getString(profile_config + "." + key));
profile->elements.emplace_back(std::move(profile_element));
}

View File

@ -0,0 +1,90 @@
#pragma once
#include <Core/Settings.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
//
// Settings from different classes (Settings, MergeTreeSettings) can coexist in the same "namespace".
// This is, for example, required to define settings constraints inside user profiles.
// `resolveSetting(full_name)` is used to resolve setting name and choose which class is to be used.
// Templated lambda syntax should be used:
//
// return resolveSetting(name, [] <typename T> (std::string_view name, SettingsType<T>)
// {
// return T::castValueUtil(name, value); // T will be deduced into `Settings`, `MergeTreeSettings`, ...
// });
//
namespace DB
{
constexpr std::string_view MERGE_TREE_SETTINGS_PREFIX = "merge_tree_";
template <typename T> struct SettingsType {};
// Resolve setting name and call function `f` back with short name and class
template <typename F>
auto resolveSetting(std::string_view full_name, F && f)
{
if (full_name.starts_with(MERGE_TREE_SETTINGS_PREFIX))
{
std::string_view short_name = static_cast<std::string_view>(full_name).substr(MERGE_TREE_SETTINGS_PREFIX.size());
if (MergeTreeSettings::hasBuiltin(short_name)) // Check is required because `Settings` also contain names starting with 'merge_tree_' prefix
return f(short_name, SettingsType<MergeTreeSettings>());
}
// NOTE: other setting name resolution rules are to be added here
// If no rule works - use global namespace
return f(full_name, SettingsType<Settings>());
}
inline Field settingCastValueUtil(std::string_view full_name, const Field & value)
{
return resolveSetting(full_name, [&] <typename T> (std::string_view short_name, SettingsType<T>)
{
return T::castValueUtil(short_name, value);
});
}
inline String settingValueToStringUtil(std::string_view full_name, const Field & value)
{
return resolveSetting(full_name, [&] <typename T> (std::string_view short_name, SettingsType<T>)
{
return T::valueToStringUtil(short_name, value);
});
}
inline Field settingStringToValueUtil(std::string_view full_name, const String & str)
{
return resolveSetting(full_name, [&] <typename T> (std::string_view short_name, SettingsType<T>)
{
return T::stringToValueUtil(short_name, str);
});
}
inline bool settingIsBuiltin(std::string_view full_name)
{
return resolveSetting(full_name, [&] <typename T> (std::string_view short_name, SettingsType<T>)
{
return T::hasBuiltin(short_name);
});
}
template <typename T>
inline String settingFullName(std::string_view short_name);
template <>
inline String settingFullName<Settings>(std::string_view short_name)
{
return String(short_name);
}
template <>
inline String settingFullName<MergeTreeSettings>(std::string_view short_name)
{
String full_name(MERGE_TREE_SETTINGS_PREFIX);
full_name += short_name; // Just because you cannot concatenate `std::string_view` and `std::string` using operator+ in C++20 yet
return full_name;
}
}

View File

@ -1431,6 +1431,11 @@ void Context::clampToSettingsConstraints(SettingsChanges & changes) const
getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes);
}
void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const
{
getSettingsConstraintsAndCurrentProfiles()->constraints.check(merge_tree_settings, changes);
}
void Context::resetSettingsToDefaultValue(const std::vector<String> & names)
{
auto lock = getLock();

View File

@ -662,6 +662,7 @@ public:
void checkSettingsConstraints(const SettingsChanges & changes) const;
void checkSettingsConstraints(SettingsChanges & changes) const;
void clampToSettingsConstraints(SettingsChanges & changes) const;
void checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const;
/// Reset settings to default value
void resetSettingsToDefaultValue(const std::vector<String> & names);

View File

@ -2665,6 +2665,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
{
const auto current_changes = old_metadata.getSettingsChanges()->as<const ASTSetQuery &>().changes;
const auto & new_changes = new_metadata.settings_changes->as<const ASTSetQuery &>().changes;
local_context->checkMergeTreeSettingsConstraints(*settings_from_storage, new_changes);
for (const auto & changed_setting : new_changes)
{
const auto & setting_name = changed_setting.name;

View File

@ -517,11 +517,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
metadata.setColumns(columns);
metadata.setComment(args.comment);
std::unique_ptr<MergeTreeSettings> storage_settings;
if (replicated)
storage_settings = std::make_unique<MergeTreeSettings>(args.getContext()->getReplicatedMergeTreeSettings());
else
storage_settings = std::make_unique<MergeTreeSettings>(args.getContext()->getMergeTreeSettings());
const auto & initial_storage_settings = replicated ? args.getContext()->getReplicatedMergeTreeSettings() : args.getContext()->getMergeTreeSettings();
std::unique_ptr<MergeTreeSettings> storage_settings = std::make_unique<MergeTreeSettings>(initial_storage_settings);
if (is_extended_storage_def)
{
@ -609,7 +606,11 @@ static StoragePtr create(const StorageFactory::Arguments & args)
// updates the default storage_settings with settings specified via SETTINGS arg in a query
if (args.storage_def->settings)
{
if (!args.attach)
args.getLocalContext()->checkMergeTreeSettingsConstraints(initial_storage_settings, args.storage_def->settings->changes);
metadata.settings_changes = args.storage_def->settings->ptr();
}
}
else
{
@ -658,7 +659,15 @@ static StoragePtr create(const StorageFactory::Arguments & args)
const auto * ast = engine_args[arg_num]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::UInt64)
{
storage_settings->index_granularity = ast->value.safeGet<UInt64>();
if (!args.attach)
{
SettingsChanges changes;
changes.emplace_back("index_granularity", Field(storage_settings->index_granularity));
args.getLocalContext()->checkMergeTreeSettingsConstraints(initial_storage_settings, changes);
}
}
else
throw Exception(
"Index granularity must be a positive integer" + getMergeTreeVerboseHelp(is_extended_storage_def),

View File

@ -1,7 +1,9 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemMergeTreeSettings.h>
#include <Access/SettingsConstraintsAndProfileIDs.h>
namespace DB
@ -15,6 +17,9 @@ NamesAndTypesList SystemMergeTreeSettings<replicated>::getNamesAndTypes()
{"value", std::make_shared<DataTypeString>()},
{"changed", std::make_shared<DataTypeUInt8>()},
{"description", std::make_shared<DataTypeString>()},
{"min", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"max", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"readonly", std::make_shared<DataTypeUInt8>()},
{"type", std::make_shared<DataTypeString>()},
};
}
@ -22,14 +27,31 @@ NamesAndTypesList SystemMergeTreeSettings<replicated>::getNamesAndTypes()
template <bool replicated>
void SystemMergeTreeSettings<replicated>::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
const auto & settings = replicated ? context->getReplicatedMergeTreeSettings().all() : context->getMergeTreeSettings().all();
for (const auto & setting : settings)
const auto & settings = replicated ? context->getReplicatedMergeTreeSettings() : context->getMergeTreeSettings();
auto constraints_and_current_profiles = context->getSettingsConstraintsAndCurrentProfiles();
const auto & constraints = constraints_and_current_profiles->constraints;
for (const auto & setting : settings.all())
{
res_columns[0]->insert(setting.getName());
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());
res_columns[4]->insert(setting.getTypeName());
Field min, max;
SettingConstraintWritability writability = SettingConstraintWritability::WRITABLE;
constraints.get(settings, setting_name, min, max, writability);
/// These two columns can accept strings only.
if (!min.isNull())
min = Settings::valueToStringUtil(setting_name, min);
if (!max.isNull())
max = Settings::valueToStringUtil(setting_name, max);
res_columns[4]->insert(min);
res_columns[5]->insert(max);
res_columns[6]->insert(writability == SettingConstraintWritability::CONST);
res_columns[7]->insert(setting.getTypeName());
}
}

View File

@ -0,0 +1,48 @@
import pytest
import asyncio
import re
import random
import os.path
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry, TSV
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance("instance", user_configs=["users.xml"])
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_merge_tree_settings_constraints():
assert "Setting storage_policy should not be changed" in instance.query_and_get_error(
f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'"
)
expected_error = "Setting min_bytes_for_wide_part should"
assert expected_error in instance.query_and_get_error(
f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS min_bytes_for_wide_part = 100"
)
assert expected_error in instance.query_and_get_error(
f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS min_bytes_for_wide_part = 1000000000"
)
instance.query(
f"CREATE TABLE good_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS min_bytes_for_wide_part = 10000000"
)
assert expected_error in instance.query_and_get_error(
f"ALTER TABLE good_table MODIFY SETTING min_bytes_for_wide_part = 100"
)
assert expected_error in instance.query_and_get_error(
f"ALTER TABLE good_table MODIFY SETTING min_bytes_for_wide_part = 1000000000"
)

View File

@ -0,0 +1,31 @@
<clickhouse>
<profiles>
<default>
<constraints>
<merge_tree_min_bytes_for_wide_part>
<min>1000</min>
<max>100000000</max>
</merge_tree_min_bytes_for_wide_part>
<merge_tree_storage_policy>
<const/>
</merge_tree_storage_policy>
</constraints>
</default>
<readonly>
<readonly>1</readonly>
</readonly>
</profiles>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
</default>
</users>
</clickhouse>

View File

@ -1,4 +1,4 @@
send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds
storage_policy default 0 Name of storage disk policy String
storage_policy default 0 Name of storage disk policy \N \N 0 String
1
1

View File

@ -331,6 +331,9 @@ CREATE TABLE system.merge_tree_settings
`value` String,
`changed` UInt8,
`description` String,
`min` Nullable(String),
`max` Nullable(String),
`readonly` UInt8,
`type` String
)
ENGINE = SystemMergeTreeSettings
@ -890,6 +893,9 @@ CREATE TABLE system.replicated_merge_tree_settings
`value` String,
`changed` UInt8,
`description` String,
`min` Nullable(String),
`max` Nullable(String),
`readonly` UInt8,
`type` String
)
ENGINE = SystemReplicatedMergeTreeSettings