mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Intermediate refactoring
This commit is contained in:
parent
106b9717cd
commit
808f4d0b8a
@ -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.
|
||||
*/
|
||||
|
||||
/// M (mutable) for normal settings, IM (immutable) for not updateable settings.
|
||||
#define LIST_OF_SETTINGS(M, IM) \
|
||||
#define LIST_OF_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.") \
|
||||
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") \
|
||||
|
@ -17,11 +17,6 @@ class Field;
|
||||
class ReadBuffer;
|
||||
class WriteBuffer;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int IMMUTABLE_SETTING;
|
||||
}
|
||||
|
||||
/** 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.
|
||||
@ -34,7 +29,6 @@ struct SettingNumber
|
||||
{
|
||||
Type value;
|
||||
bool changed = false;
|
||||
bool immutable = false;
|
||||
|
||||
SettingNumber(Type x = 0) : value(x) {}
|
||||
|
||||
@ -77,7 +71,6 @@ struct SettingMaxThreads
|
||||
UInt64 value;
|
||||
bool is_auto;
|
||||
bool changed = false;
|
||||
bool immutable = false;
|
||||
|
||||
SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {}
|
||||
|
||||
@ -106,7 +99,6 @@ struct SettingTimespan
|
||||
{
|
||||
Poco::Timespan value;
|
||||
bool changed = false;
|
||||
bool immutable = false;
|
||||
|
||||
SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {}
|
||||
|
||||
@ -139,7 +131,6 @@ struct SettingString
|
||||
{
|
||||
String value;
|
||||
bool changed = false;
|
||||
bool immutable = false;
|
||||
|
||||
SettingString(const String & x = String{}) : value(x) {}
|
||||
|
||||
@ -162,7 +153,6 @@ struct SettingChar
|
||||
public:
|
||||
char value;
|
||||
bool changed = false;
|
||||
bool immutable = false;
|
||||
|
||||
SettingChar(char x = '\0') : value(x) {}
|
||||
|
||||
@ -187,7 +177,6 @@ struct SettingEnum
|
||||
{
|
||||
EnumType value;
|
||||
bool changed = false;
|
||||
bool immutable = false;
|
||||
|
||||
SettingEnum(EnumType x) : value(x) {}
|
||||
|
||||
@ -322,8 +311,6 @@ private:
|
||||
using SerializeFunction = void (*)(const Derived &, WriteBuffer & buf);
|
||||
using DeserializeFunction = void (*)(Derived &, ReadBuffer & buf);
|
||||
using CastValueWithoutApplyingFunction = Field (*)(const Field &);
|
||||
using SetImmutable = void(*)(Derived &);
|
||||
using IsImmutable = bool(*)(const Derived &);
|
||||
|
||||
|
||||
struct MemberInfo
|
||||
@ -331,8 +318,6 @@ private:
|
||||
IsChangedFunction is_changed;
|
||||
StringRef name;
|
||||
StringRef description;
|
||||
/// At one moment this setting can became immutable
|
||||
const bool can_be_immutable;
|
||||
GetStringFunction get_string;
|
||||
GetFieldFunction get_field;
|
||||
SetStringFunction set_string;
|
||||
@ -340,8 +325,6 @@ private:
|
||||
SerializeFunction serialize;
|
||||
DeserializeFunction deserialize;
|
||||
CastValueWithoutApplyingFunction cast_value_without_applying;
|
||||
SetImmutable set_immutable;
|
||||
IsImmutable is_immutable;
|
||||
|
||||
bool isChanged(const Derived & collection) const { return is_changed(collection); }
|
||||
};
|
||||
@ -431,20 +414,8 @@ public:
|
||||
public:
|
||||
reference(Derived & collection_, const MemberInfo & member_) : const_reference(collection_, member_) {}
|
||||
reference(const const_reference & src) : const_reference(src) {}
|
||||
void setValue(const Field & 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)); }
|
||||
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.
|
||||
@ -615,14 +586,6 @@ public:
|
||||
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).
|
||||
/// Only changed settings are written. They are written as list of contiguous name-value pairs,
|
||||
/// 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_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##_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 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) \
|
||||
@ -693,14 +654,4 @@ public:
|
||||
&Functions::NAME##_setString, &Functions::NAME##_setField, \
|
||||
&Functions::NAME##_serialize, &Functions::NAME##_deserialize, \
|
||||
&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 });
|
||||
}
|
||||
|
@ -543,15 +543,8 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
}
|
||||
}
|
||||
else if (command.type == AlterCommand::MODIFY_SETTING)
|
||||
{
|
||||
for (const auto & change : command.settings_changes)
|
||||
{
|
||||
if (!table.hasSetting(change.name))
|
||||
{
|
||||
throw Exception{"Storage '" + table.getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING};
|
||||
}
|
||||
}
|
||||
}
|
||||
table.checkSetting(change.name);
|
||||
}
|
||||
|
||||
/** Existing defaulted columns may require default expression extensions with a type conversion,
|
||||
|
@ -308,11 +308,10 @@ bool IStorage::isVirtualColumn(const String & column_name) const
|
||||
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())
|
||||
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)
|
||||
@ -380,16 +379,13 @@ IDatabase::ASTModifier IStorage::getSettingsModifier(const SettingsChanges & new
|
||||
/// Make storage settings unique
|
||||
for (const auto & change : new_changes)
|
||||
{
|
||||
if (hasSetting(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())
|
||||
it->value = change.value;
|
||||
else
|
||||
storage_changes.push_back(change);
|
||||
}
|
||||
checkSetting(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())
|
||||
it->value = change.value;
|
||||
else
|
||||
throw Exception{"Storage '" + getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING};
|
||||
storage_changes.push_back(change);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -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.
|
||||
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.
|
||||
virtual bool hasSetting(const String & setting_name) const;
|
||||
/// Check storage has setting and setting can be modified.
|
||||
virtual bool checkSetting(const String & setting_name) const;
|
||||
|
||||
protected: /// still thread-unsafe part.
|
||||
void setIndices(IndicesDescription indices_);
|
||||
@ -149,7 +149,7 @@ protected: /// still thread-unsafe part.
|
||||
virtual bool isVirtualColumn(const String & column_name) const;
|
||||
|
||||
/// 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:
|
||||
ColumnsDescription columns; /// combined real and virtual columns
|
||||
|
@ -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, IM) \
|
||||
IM(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \
|
||||
IM(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \
|
||||
IM(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \
|
||||
IM(SettingString, kafka_format, "", "The message format for Kafka engine.") \
|
||||
IM(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \
|
||||
IM(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \
|
||||
IM(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \
|
||||
IM(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \
|
||||
IM(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \
|
||||
IM(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block")
|
||||
#define LIST_OF_KAFKA_SETTINGS(M) \
|
||||
M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \
|
||||
M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \
|
||||
M(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \
|
||||
M(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.") \
|
||||
M(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.") \
|
||||
M(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") \
|
||||
M(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)
|
||||
|
||||
|
@ -34,16 +34,37 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event RejectedInserts;
|
||||
extern const Event DelayedInserts;
|
||||
extern const Event DelayedInsertsMilliseconds;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DelayedInserts;
|
||||
}
|
||||
|
||||
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 NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
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
|
||||
@ -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("Storage '" + getName() + "' doesn't support settings alter", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
throw Exception{"Setting '" + setting_name + "' is immutable for storage '" + getName() + "'", ErrorCodes::IMMUTABLE_SETTING};
|
||||
}
|
||||
|
||||
void registerStorageKafka(StorageFactory & factory)
|
||||
|
@ -57,8 +57,7 @@ public:
|
||||
const auto & getSchemaName() const { return schema_name; }
|
||||
const auto & skipBroken() const { return skip_broken; }
|
||||
|
||||
bool hasSetting(const String & setting_name) const override;
|
||||
|
||||
void checkSetting(const String & setting_name) const override;
|
||||
|
||||
protected:
|
||||
StorageKafka(
|
||||
@ -71,7 +70,6 @@ protected:
|
||||
size_t num_consumers_, UInt64 max_block_size_, size_t skip_broken,
|
||||
bool intermediate_commit_);
|
||||
|
||||
IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const override;
|
||||
private:
|
||||
// Configuration and state
|
||||
String table_name;
|
||||
|
@ -91,6 +91,7 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int BAD_DATA_PART_NAME;
|
||||
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);
|
||||
|
||||
for (const auto & setting : new_changes)
|
||||
{
|
||||
if (!hasSetting(setting.name))
|
||||
throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting.name + "'", ErrorCodes::UNKNOWN_SETTING};
|
||||
}
|
||||
checkSetting(setting.name);
|
||||
|
||||
/// Check that type conversions are possible.
|
||||
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)
|
||||
|
@ -543,7 +543,7 @@ public:
|
||||
TableStructureWriteLockHolder & table_lock_holder);
|
||||
|
||||
/// 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
|
||||
void removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part);
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/SettingsCommon.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
@ -24,9 +25,8 @@ class ASTStorage;
|
||||
struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
|
||||
{
|
||||
|
||||
/// M (mutable) for normal settings, IM (immutable) for not updateable settings.
|
||||
#define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \
|
||||
IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \
|
||||
#define LIST_OF_MERGE_TREE_SETTINGS(M) \
|
||||
M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \
|
||||
\
|
||||
/** 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).") \
|
||||
@ -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(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).") \
|
||||
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(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") \
|
||||
@ -98,6 +98,12 @@ struct MergeTreeSettings : public SettingsCollection<MergeTreeSettings>
|
||||
|
||||
/// NOTE: will rewrite the AST to add immutable settings.
|
||||
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>;
|
||||
|
@ -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 " \
|
||||
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/// Finalize settings and disable updates
|
||||
storage_settings->finishSettingsInitialization();
|
||||
|
||||
if (replicated)
|
||||
return StorageReplicatedMergeTree::create(
|
||||
zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name,
|
||||
|
Loading…
Reference in New Issue
Block a user