Intermediate refactoring

This commit is contained in:
alesapin 2019-08-30 19:50:59 +03:00
parent 106b9717cd
commit 808f4d0b8a
12 changed files with 78 additions and 118 deletions

View File

@ -42,8 +42,7 @@ struct Settings : public SettingsCollection<Settings>
* but we are not going to do it, because settings is used everywhere as static struct fields. * but we are not going to do it, because settings is used everywhere as static struct fields.
*/ */
/// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_SETTINGS(M) \
#define LIST_OF_SETTINGS(M, IM) \
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.") \ 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.") \
M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \ M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \
M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading") \ M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading") \

View File

@ -17,11 +17,6 @@ class Field;
class ReadBuffer; class ReadBuffer;
class WriteBuffer; class WriteBuffer;
namespace ErrorCodes
{
extern const int IMMUTABLE_SETTING;
}
/** One setting for any type. /** One setting for any type.
* Stores a value within itself, as well as a flag - whether the value was changed. * 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. * This is done so that you can send to the remote servers only changed settings (or explicitly specified in the config) values.
@ -34,7 +29,6 @@ struct SettingNumber
{ {
Type value; Type value;
bool changed = false; bool changed = false;
bool immutable = false;
SettingNumber(Type x = 0) : value(x) {} SettingNumber(Type x = 0) : value(x) {}
@ -77,7 +71,6 @@ struct SettingMaxThreads
UInt64 value; UInt64 value;
bool is_auto; bool is_auto;
bool changed = false; bool changed = false;
bool immutable = false;
SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {}
@ -106,7 +99,6 @@ struct SettingTimespan
{ {
Poco::Timespan value; Poco::Timespan value;
bool changed = false; bool changed = false;
bool immutable = false;
SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {}
@ -139,7 +131,6 @@ struct SettingString
{ {
String value; String value;
bool changed = false; bool changed = false;
bool immutable = false;
SettingString(const String & x = String{}) : value(x) {} SettingString(const String & x = String{}) : value(x) {}
@ -162,7 +153,6 @@ struct SettingChar
public: public:
char value; char value;
bool changed = false; bool changed = false;
bool immutable = false;
SettingChar(char x = '\0') : value(x) {} SettingChar(char x = '\0') : value(x) {}
@ -187,7 +177,6 @@ struct SettingEnum
{ {
EnumType value; EnumType value;
bool changed = false; bool changed = false;
bool immutable = false;
SettingEnum(EnumType x) : value(x) {} SettingEnum(EnumType x) : value(x) {}
@ -322,8 +311,6 @@ private:
using SerializeFunction = void (*)(const Derived &, WriteBuffer & buf); using SerializeFunction = void (*)(const Derived &, WriteBuffer & buf);
using DeserializeFunction = void (*)(Derived &, ReadBuffer & buf); using DeserializeFunction = void (*)(Derived &, ReadBuffer & buf);
using CastValueWithoutApplyingFunction = Field (*)(const Field &); using CastValueWithoutApplyingFunction = Field (*)(const Field &);
using SetImmutable = void(*)(Derived &);
using IsImmutable = bool(*)(const Derived &);
struct MemberInfo struct MemberInfo
@ -331,8 +318,6 @@ private:
IsChangedFunction is_changed; IsChangedFunction is_changed;
StringRef name; StringRef name;
StringRef description; StringRef description;
/// At one moment this setting can became immutable
const bool can_be_immutable;
GetStringFunction get_string; GetStringFunction get_string;
GetFieldFunction get_field; GetFieldFunction get_field;
SetStringFunction set_string; SetStringFunction set_string;
@ -340,8 +325,6 @@ private:
SerializeFunction serialize; SerializeFunction serialize;
DeserializeFunction deserialize; DeserializeFunction deserialize;
CastValueWithoutApplyingFunction cast_value_without_applying; CastValueWithoutApplyingFunction cast_value_without_applying;
SetImmutable set_immutable;
IsImmutable is_immutable;
bool isChanged(const Derived & collection) const { return is_changed(collection); } bool isChanged(const Derived & collection) const { return is_changed(collection); }
}; };
@ -431,20 +414,8 @@ public:
public: public:
reference(Derived & collection_, const MemberInfo & member_) : const_reference(collection_, member_) {} reference(Derived & collection_, const MemberInfo & member_) : const_reference(collection_, member_) {}
reference(const const_reference & src) : const_reference(src) {} reference(const const_reference & src) : const_reference(src) {}
void setValue(const Field & value) 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); }
if (this->member->is_immutable(*this->collection))
throw Exception("Setting '" + this->member->name.toString() + "' is restricted for updates.", ErrorCodes::IMMUTABLE_SETTING);
this->member->set_field(*const_cast<Derived *>(this->collection), value);
}
void setValue(const String & value)
{
if (this->member->is_immutable(*this->collection))
throw Exception("Setting '" + this->member->name.toString() + "' is restricted for updates.", ErrorCodes::IMMUTABLE_SETTING);
this->member->set_string(*const_cast<Derived *>(this->collection), value);
}
bool canBeImmutable() const { return this->member->can_be_immutable; }
void makeImmutableForever() { this->member->set_immutable(*const_cast<Derived *>(this->collection)); }
}; };
/// Iterator to iterating through all the settings. /// Iterator to iterating through all the settings.
@ -615,14 +586,6 @@ public:
dest.copyChangesFrom(castToDerived()); dest.copyChangesFrom(castToDerived());
} }
/// Make all possible immutable settings (can_be_immutable) immutable forever
void finishSettingsInitialization()
{
for (auto & member : *this)
if (member.canBeImmutable())
member.makeImmutableForever();
}
/// Writes the settings to buffer (e.g. to be sent to remote server). /// 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, /// Only changed settings are written. They are written as list of contiguous name-value pairs,
/// finished with empty name. /// finished with empty name.
@ -666,7 +629,7 @@ public:
{ \ { \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_, IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_, IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \
}; \ }; \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_, IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_) \ LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_) \
} }
@ -682,8 +645,6 @@ public:
static void NAME##_serialize(const Derived & collection, WriteBuffer & buf) { collection.NAME.serialize(buf); } \ static void NAME##_serialize(const Derived & collection, WriteBuffer & buf) { collection.NAME.serialize(buf); } \
static void NAME##_deserialize(Derived & collection, ReadBuffer & buf) { collection.NAME.deserialize(buf); } \ static void NAME##_deserialize(Derived & collection, ReadBuffer & buf) { collection.NAME.deserialize(buf); } \
static Field NAME##_castValueWithoutApplying(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ static Field NAME##_castValueWithoutApplying(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \
static void NAME##_setImmutable(Derived & collection) { collection.NAME.immutable = true; } \
static bool NAME##_isImmutable(const Derived & collection) { return collection.NAME.immutable; }
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \
@ -693,14 +654,4 @@ public:
&Functions::NAME##_setString, &Functions::NAME##_setField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \
&Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \
&Functions::NAME##_castValueWithoutApplying, \ &Functions::NAME##_castValueWithoutApplying, \
&Functions::NAME##_setImmutable, &Functions::NAME##_isImmutable });
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \
add({[](const Derived & d) { return d.NAME.changed; }, \
StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \
&Functions::NAME##_getString, &Functions::NAME##_getField, \
&Functions::NAME##_setString, &Functions::NAME##_setField, \
&Functions::NAME##_serialize, &Functions::NAME##_deserialize, \
&Functions::NAME##_castValueWithoutApplying, \
&Functions::NAME##_setImmutable, &Functions::NAME##_isImmutable });
} }

View File

@ -543,15 +543,8 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
} }
} }
else if (command.type == AlterCommand::MODIFY_SETTING) else if (command.type == AlterCommand::MODIFY_SETTING)
{
for (const auto & change : command.settings_changes) for (const auto & change : command.settings_changes)
{ table.checkSetting(change.name);
if (!table.hasSetting(change.name))
{
throw Exception{"Storage '" + table.getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING};
}
}
}
} }
/** Existing defaulted columns may require default expression extensions with a type conversion, /** Existing defaulted columns may require default expression extensions with a type conversion,

View File

@ -308,11 +308,10 @@ bool IStorage::isVirtualColumn(const String & column_name) const
return getColumns().get(column_name).is_virtual; return getColumns().get(column_name).is_virtual;
} }
bool IStorage::hasSetting(const String & /* setting_name */) const void IStorage::checkSetting(const String & /* setting_name */) const
{ {
if (!supportsSettings()) if (!supportsSettings())
throw Exception("Storage '" + getName() + "' doesn't support settings.", ErrorCodes::SETTINGS_ARE_NOT_SUPPORTED); throw Exception("Storage '" + getName() + "' doesn't support settings.", ErrorCodes::SETTINGS_ARE_NOT_SUPPORTED);
return false;
} }
TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id) TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id)
@ -380,16 +379,13 @@ IDatabase::ASTModifier IStorage::getSettingsModifier(const SettingsChanges & new
/// Make storage settings unique /// Make storage settings unique
for (const auto & change : new_changes) for (const auto & change : new_changes)
{ {
if (hasSetting(change.name)) checkSetting(change.name);
{
auto finder = [&change] (const SettingChange & c) { return c.name == change.name; }; auto finder = [&change] (const SettingChange & c) { return c.name == change.name; };
if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end()) if (auto it = std::find_if(storage_changes.begin(), storage_changes.end(), finder); it != storage_changes.end())
it->value = change.value; it->value = change.value;
else
storage_changes.push_back(change);
}
else else
throw Exception{"Storage '" + getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING}; storage_changes.push_back(change);
} }
} }
}; };

View File

@ -138,8 +138,8 @@ public: /// thread-unsafe part. lockStructure must be acquired
/// If |need_all| is set, then checks that all the columns of the table are in the block. /// If |need_all| is set, then checks that all the columns of the table are in the block.
void check(const Block & block, bool need_all = false) const; void check(const Block & block, bool need_all = false) const;
/// Check storage has setting. Exception will be thrown if it doesn't support settings at all. /// Check storage has setting and setting can be modified.
virtual bool hasSetting(const String & setting_name) const; virtual bool checkSetting(const String & setting_name) const;
protected: /// still thread-unsafe part. protected: /// still thread-unsafe part.
void setIndices(IndicesDescription indices_); void setIndices(IndicesDescription indices_);
@ -149,7 +149,7 @@ protected: /// still thread-unsafe part.
virtual bool isVirtualColumn(const String & column_name) const; virtual bool isVirtualColumn(const String & column_name) const;
/// Returns modifier of settings in storage definition /// Returns modifier of settings in storage definition
virtual IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const;
private: private:
ColumnsDescription columns; /// combined real and virtual columns ColumnsDescription columns; /// combined real and virtual columns

View File

@ -15,18 +15,17 @@ struct KafkaSettings : public SettingsCollection<KafkaSettings>
{ {
/// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_KAFKA_SETTINGS(M) \
#define LIST_OF_KAFKA_SETTINGS(M, IM) \ M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \
IM(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \
IM(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ M(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \
IM(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ M(SettingString, kafka_format, "", "The message format for Kafka engine.") \
IM(SettingString, kafka_format, "", "The message format for Kafka engine.") \ M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \
IM(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \
IM(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \
IM(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \
IM(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \
IM(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ M(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block")
IM(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block")
DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS) DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS)

View File

@ -34,16 +34,37 @@
namespace DB namespace DB
{ {
namespace ProfileEvents
{
extern const Event RejectedInserts;
extern const Event DelayedInserts;
extern const Event DelayedInsertsMilliseconds;
}
namespace CurrentMetrics
{
extern const Metric DelayedInserts;
}
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int INCORRECT_DATA;
extern const int UNKNOWN_EXCEPTION;
extern const int CANNOT_READ_FROM_ISTREAM;
extern const int INVALID_CONFIG_PARAMETER;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int MEMORY_LIMIT_EXCEEDED;
extern const int UNSUPPORTED_METHOD; extern const int SYNTAX_ERROR;
extern const int INVALID_PARTITION_VALUE;
extern const int METADATA_MISMATCH;
extern const int PART_IS_TEMPORARILY_LOCKED;
extern const int TOO_MANY_PARTS;
extern const int INCOMPATIBLE_COLUMNS;
extern const int CANNOT_UPDATE_COLUMN;
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int CANNOT_MUNMAP;
extern const int CANNOT_MREMAP;
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 IMMUTABLE_SETTING;
} }
namespace namespace
@ -407,14 +428,12 @@ bool StorageKafka::streamToViews()
} }
bool StorageKafka::hasSetting(const String & setting_name) const void StorageKafka::checkSetting(const String & setting_name) const
{ {
return KafkaSettings::findIndex(setting_name) != KafkaSettings::npos; if (KafkaSettings::findIndex(setting_name) == KafkaSettings::npos)
} throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting_name + "'", ErrorCodes::UNKNOWN_SETTING};
IDatabase::ASTModifier StorageKafka::getSettingsModifier(const SettingsChanges & /* new_changes */) const throw Exception{"Setting '" + setting_name + "' is immutable for storage '" + getName() + "'", ErrorCodes::IMMUTABLE_SETTING};
{
throw Exception("Storage '" + getName() + "' doesn't support settings alter", ErrorCodes::UNSUPPORTED_METHOD);
} }
void registerStorageKafka(StorageFactory & factory) void registerStorageKafka(StorageFactory & factory)

View File

@ -57,8 +57,7 @@ public:
const auto & getSchemaName() const { return schema_name; } const auto & getSchemaName() const { return schema_name; }
const auto & skipBroken() const { return skip_broken; } const auto & skipBroken() const { return skip_broken; }
bool hasSetting(const String & setting_name) const override; void checkSetting(const String & setting_name) const override;
protected: protected:
StorageKafka( StorageKafka(
@ -71,7 +70,6 @@ protected:
size_t num_consumers_, UInt64 max_block_size_, size_t skip_broken, size_t num_consumers_, UInt64 max_block_size_, size_t skip_broken,
bool intermediate_commit_); bool intermediate_commit_);
IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const override;
private: private:
// Configuration and state // Configuration and state
String table_name; String table_name;

View File

@ -91,6 +91,7 @@ namespace ErrorCodes
extern const int INCORRECT_FILE_NAME; extern const int INCORRECT_FILE_NAME;
extern const int BAD_DATA_PART_NAME; extern const int BAD_DATA_PART_NAME;
extern const int UNKNOWN_SETTING; extern const int UNKNOWN_SETTING;
extern const int IMMUTABLE_SETTING;
} }
@ -1324,10 +1325,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true); setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true);
for (const auto & setting : new_changes) for (const auto & setting : new_changes)
{ checkSetting(setting.name);
if (!hasSetting(setting.name))
throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting.name + "'", ErrorCodes::UNKNOWN_SETTING};
}
/// Check that type conversions are possible. /// Check that type conversions are possible.
ExpressionActionsPtr unused_expression; ExpressionActionsPtr unused_expression;
@ -1657,9 +1655,13 @@ void MergeTreeData::changeSettings(
} }
} }
bool MergeTreeData::hasSetting(const String & setting_name) const void MergeTreeData::checkSetting(const String & setting_name) const
{ {
return MergeTreeSettings::findIndex(setting_name) != MergeTreeSettings::npos; if (MergeTreeSettings::findIndex(setting_name) == MergeTreeSettings::npos)
throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting_name + "'", ErrorCodes::UNKNOWN_SETTING};
if (MergeTreeSettings::isImmutableSetting(setting_name))
throw Exception{"Setting '" + setting_name + "' is immutable for storage '" + getName() + "'", ErrorCodes::IMMUTABLE_SETTING};
} }
void MergeTreeData::removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part) void MergeTreeData::removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part)

View File

@ -543,7 +543,7 @@ public:
TableStructureWriteLockHolder & table_lock_holder); TableStructureWriteLockHolder & table_lock_holder);
/// All MergeTreeData children have settings. /// All MergeTreeData children have settings.
bool hasSetting(const String & setting_name) const override; void checkSetting(const String & setting_name) const override;
/// Remove columns, that have been markedd as empty after zeroing values with expired ttl /// Remove columns, that have been markedd as empty after zeroing values with expired ttl
void removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part); void removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part);

View File

@ -2,6 +2,7 @@
#include <Core/Defines.h> #include <Core/Defines.h>
#include <Core/SettingsCommon.h> #include <Core/SettingsCommon.h>
#include <Common/SettingsChanges.h>
namespace Poco namespace Poco
@ -24,9 +25,8 @@ class ASTStorage;
struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings> struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
{ {
/// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_MERGE_TREE_SETTINGS(M) \
#define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \ M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \
IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \
\ \
/** Merge settings. */ \ /** Merge settings. */ \
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).") \ 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).") \
@ -80,7 +80,7 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
M(SettingBool, use_minimalistic_part_header_in_zookeeper, false, "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.") \ M(SettingBool, use_minimalistic_part_header_in_zookeeper, false, "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.") \
M(SettingUInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.") \ M(SettingUInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.") \
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).") \ 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).") \
IM(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).") \ M(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).") \
M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.") \ M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.") \
M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)") \ M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)") \
M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity") \ M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity") \
@ -98,6 +98,12 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
/// NOTE: will rewrite the AST to add immutable settings. /// NOTE: will rewrite the AST to add immutable settings.
void loadFromQuery(ASTStorage & storage_def); void loadFromQuery(ASTStorage & storage_def);
/// We check settings after storage creation
static bool isImmutableSetting(const String & name)
{
return name == "index_granularity" || name == "index_granularity_bytes";
}
}; };
using MergeTreeSettingsPtr = std::shared_ptr<const MergeTreeSettings>; using MergeTreeSettingsPtr = std::shared_ptr<const MergeTreeSettings>;

View File

@ -637,9 +637,6 @@ static StoragePtr create(const StorageFactory::Arguments & args)
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \ throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS); "before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
/// Finalize settings and disable updates
storage_settings->finishSettingsInitialization();
if (replicated) if (replicated)
return StorageReplicatedMergeTree::create( return StorageReplicatedMergeTree::create(
zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name,