mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add some changes
This commit is contained in:
parent
86831fe500
commit
e12cb8da6d
@ -321,7 +321,7 @@ private:
|
||||
size_t offset_of_changed;
|
||||
StringRef name;
|
||||
StringRef description;
|
||||
bool immutable;
|
||||
bool updateable;
|
||||
GetStringFunction get_string;
|
||||
GetFieldFunction get_field;
|
||||
SetStringFunction set_string;
|
||||
@ -401,7 +401,7 @@ public:
|
||||
const_reference(const const_reference & src) = default;
|
||||
const StringRef & getName() const { return member->name; }
|
||||
const StringRef & getDescription() const { return member->description; }
|
||||
bool isImmutable() const { return member->immutable; }
|
||||
bool isUpdateable() const { return member->updateable; }
|
||||
bool isChanged() const { return member->isChanged(*collection); }
|
||||
Field getValue() const { return member->get_field(*collection); }
|
||||
String getValueAsString() const { return member->get_string(*collection); }
|
||||
@ -421,6 +421,18 @@ public:
|
||||
reference(const const_reference & src) : const_reference(src) {}
|
||||
void setValue(const Field & value) { this->member->set_field(*const_cast<Derived *>(this->collection), value); }
|
||||
void setValue(const String & value) { this->member->set_string(*const_cast<Derived *>(this->collection), value); }
|
||||
void updateValue(const Field & value)
|
||||
{
|
||||
if (!this->member->updateable)
|
||||
throw Exception("Setting '" + this->member->name.toString() + "' is restricted for updates.", ErrorCodes::IMMUTABLE_SETTING);
|
||||
setValue(value);
|
||||
}
|
||||
void updateValue(const String & value)
|
||||
{
|
||||
if (!this->member->updateable)
|
||||
throw Exception("Setting '" + this->member->name.toString() + "' is restricted for updates.", ErrorCodes::IMMUTABLE_SETTING);
|
||||
setValue(value);
|
||||
}
|
||||
};
|
||||
|
||||
/// Iterator to iterating through all the settings.
|
||||
@ -503,39 +515,14 @@ public:
|
||||
void set(size_t index, const String & value) { (*this)[index].setValue(value); }
|
||||
void set(const String & name, const String & value) { (*this)[name].setValue(value); }
|
||||
|
||||
/// Updates setting's value.
|
||||
void update(size_t index, const Field & value)
|
||||
{
|
||||
auto ref = (*this)[index];
|
||||
if (ref.isImmutable())
|
||||
throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING);
|
||||
ref.setValue(value);
|
||||
}
|
||||
/// Updates setting's value. Checks it' mutability.
|
||||
void update(size_t index, const Field & value) { (*this)[index].updateValue(value); }
|
||||
|
||||
void update(const String & name, const Field & value)
|
||||
{
|
||||
auto ref = (*this)[name];
|
||||
if (ref.isImmutable())
|
||||
throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING);
|
||||
ref.setValue(value);
|
||||
}
|
||||
void update(const String & name, const Field & value) { (*this)[name].updateValue(value); }
|
||||
|
||||
/// Updates setting's value. Read value in text form from string (for example, from configuration file or from URL parameter).
|
||||
void update(size_t index, const String & value)
|
||||
{
|
||||
auto ref = (*this)[index];
|
||||
if (ref.isImmutable())
|
||||
throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING);
|
||||
(*this)[index].setValue(value);
|
||||
}
|
||||
void update(size_t index, const String & value) { (*this)[index].updateValue(value); }
|
||||
|
||||
void update(const String & name, const String & value)
|
||||
{
|
||||
auto ref = (*this)[name];
|
||||
if (ref.isImmutable())
|
||||
throw Exception("Cannot modify immutable setting '" + ref.getName().toString() + "'", ErrorCodes::IMMUTABLE_SETTING);
|
||||
(*this)[name].setValue(value);
|
||||
}
|
||||
void update(const String & name, const String & value) { (*this)[name].updateValue(value); }
|
||||
|
||||
/// Returns value of a setting.
|
||||
Field get(size_t index) const { return (*this)[index].getValue(); }
|
||||
@ -600,7 +587,7 @@ public:
|
||||
return found_changes;
|
||||
}
|
||||
|
||||
/// Applies changes to the settings.
|
||||
/// Applies changes to the settings. Doesn't check settings mutability.
|
||||
void loadFromChange(const SettingChange & change)
|
||||
{
|
||||
set(change.name, change.value);
|
||||
@ -612,7 +599,7 @@ public:
|
||||
loadFromChange(change);
|
||||
}
|
||||
|
||||
/// Applies changes to the settings.
|
||||
/// Applies changes to the settings, checks settings mutability
|
||||
void updateFromChange(const SettingChange & change)
|
||||
{
|
||||
update(change.name, change.value);
|
||||
@ -701,7 +688,7 @@ public:
|
||||
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \
|
||||
static_assert(std::is_same_v<decltype(std::declval<Derived>().NAME.changed), bool>); \
|
||||
add({offsetof(Derived, NAME.changed), \
|
||||
StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \
|
||||
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, \
|
||||
@ -710,10 +697,9 @@ public:
|
||||
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_IMMUTABLE_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \
|
||||
static_assert(std::is_same_v<decltype(std::declval<Derived>().NAME.changed), bool>); \
|
||||
add({offsetof(Derived, NAME.changed), \
|
||||
StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), true, \
|
||||
StringRef(#NAME, strlen(#NAME)), StringRef(#DESCRIPTION, strlen(#DESCRIPTION)), false, \
|
||||
&Functions::NAME##_getString, &Functions::NAME##_getField, \
|
||||
&Functions::NAME##_setString, &Functions::NAME##_setField, \
|
||||
&Functions::NAME##_serialize, &Functions::NAME##_deserialize, \
|
||||
&Functions::NAME##_castValueWithoutApplying });
|
||||
|
||||
}
|
||||
|
@ -332,6 +332,11 @@ bool AlterCommand::isMutable() const
|
||||
return true;
|
||||
}
|
||||
|
||||
bool AlterCommand::isSettingsAlter() const
|
||||
{
|
||||
return type == MODIFY_SETTING;
|
||||
}
|
||||
|
||||
void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const
|
||||
{
|
||||
@ -567,6 +572,31 @@ void AlterCommands::applyForColumnsOnly(ColumnsDescription & columns_description
|
||||
columns_description = std::move(out_columns_description);
|
||||
}
|
||||
|
||||
|
||||
void AlterCommands::applyForSettingsOnly(SettingsChanges & changes) const
|
||||
{
|
||||
ColumnsDescription out_columns_description;
|
||||
IndicesDescription indices_description;
|
||||
ASTPtr out_order_by;
|
||||
ASTPtr out_primary_key;
|
||||
ASTPtr out_ttl_table;
|
||||
SettingsChanges out_changes;
|
||||
apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table, out_changes);
|
||||
|
||||
if (out_columns_description.begin() != out_columns_description.end())
|
||||
throw Exception("Alter modifying columns, but only settings change applied.", ErrorCodes::LOGICAL_ERROR);
|
||||
if (out_order_by)
|
||||
throw Exception("Alter modifying ORDER BY expression, but only settings change applied.", ErrorCodes::LOGICAL_ERROR);
|
||||
if (out_primary_key)
|
||||
throw Exception("Alter modifying PRIMARY KEY expression, but only settings change applied.", ErrorCodes::LOGICAL_ERROR);
|
||||
if (!indices_description.indices.empty())
|
||||
throw Exception("Alter modifying indices, but only settings change applied.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (out_ttl_table)
|
||||
throw Exception("Alter modifying TTL, but only settings change applied.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
changes = std::move(out_changes);
|
||||
}
|
||||
|
||||
bool AlterCommands::isMutable() const
|
||||
{
|
||||
for (const auto & param : *this)
|
||||
@ -578,4 +608,8 @@ bool AlterCommands::isMutable() const
|
||||
return false;
|
||||
}
|
||||
|
||||
bool AlterCommands::isSettingsAlter() const
|
||||
{
|
||||
return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter(); });
|
||||
}
|
||||
}
|
||||
|
@ -93,6 +93,9 @@ struct AlterCommand
|
||||
|
||||
/// Checks that not only metadata touched by that command
|
||||
bool isMutable() const;
|
||||
|
||||
/// checks that only settings changed by alter
|
||||
bool isSettingsAlter() const;
|
||||
};
|
||||
|
||||
class Context;
|
||||
@ -107,8 +110,12 @@ public:
|
||||
/// Used for primitive table engines, where only columns metadata can be changed
|
||||
void applyForColumnsOnly(ColumnsDescription & columns_description) const;
|
||||
|
||||
/// Apply alter commands only for settings. Exception will be thrown if any other part of table structure will be modified.
|
||||
void applyForSettingsOnly(SettingsChanges & changes) const;
|
||||
|
||||
void validate(const IStorage & table, const Context & context);
|
||||
bool isMutable() const;
|
||||
bool isSettingsAlter() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,2 @@
|
||||
CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096
|
||||
CREATE TABLE default.table_for_alter (`id` UInt64, `Data` String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1
|
@ -0,0 +1,39 @@
|
||||
DROP TABLE IF EXISTS log_for_alter;
|
||||
|
||||
CREATE TABLE log_for_alter (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = Log();
|
||||
|
||||
ALTER TABLE log_for_alter MODIFY SETTING aaa=123; -- { serverError 468 }
|
||||
|
||||
DROP TABLE IF EXISTS log_for_alter;
|
||||
|
||||
DROP TABLE IF EXISTS table_for_alter;
|
||||
|
||||
CREATE TABLE table_for_alter (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096;
|
||||
|
||||
ALTER TABLE table_for_alter MODIFY SETTING index_granularity=555; -- { serverError 469 }
|
||||
|
||||
SHOW CREATE TABLE table_for_alter;
|
||||
|
||||
ALTER TABLE table_for_alter MODIFY SETTING parts_to_throw_insert = 1, parts_to_delay_insert = 1;
|
||||
|
||||
SHOW CREATE TABLE table_for_alter;
|
||||
|
||||
INSERT INTO table_for_alter VALUES (1, '1');
|
||||
INSERT INTO table_for_alter VALUES (2, '2'); -- { serverError 252 }
|
||||
|
||||
DETACH TABLE table_for_alter;
|
||||
|
||||
ATTACH TABLE table_for_alter;
|
||||
|
||||
INSERT INTO table_for_alter VALUES (2, '2'); -- { serverError 252 }
|
||||
|
||||
ALTER TABLE table_for_alter MODIFY SETTING xxx_yyy=124; -- { serverError 115 }
|
||||
|
||||
DROP TABLE IF EXISTS table_for_alter;
|
||||
|
@ -0,0 +1,10 @@
|
||||
CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192
|
||||
4
|
||||
4
|
||||
4
|
||||
4
|
||||
6
|
||||
6
|
||||
CREATE TABLE default.replicated_table_for_alter1 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'1\') ORDER BY id SETTINGS index_granularity = 8192, use_minimalistic_part_header_in_zookeeper = 1
|
||||
CREATE TABLE default.replicated_table_for_alter2 (`id` UInt64, `Data` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/replicated_table_for_alter\', \'2\') ORDER BY id SETTINGS index_granularity = 8192, parts_to_throw_insert = 1, parts_to_delay_insert = 1
|
@ -0,0 +1,62 @@
|
||||
DROP TABLE IF EXISTS replicated_table_for_alter1;
|
||||
DROP TABLE IF EXISTS replicated_table_for_alter2;
|
||||
|
||||
CREATE TABLE replicated_table_for_alter1 (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_table_for_alter', '1') ORDER BY id;
|
||||
|
||||
CREATE TABLE replicated_table_for_alter2 (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_table_for_alter', '2') ORDER BY id;
|
||||
|
||||
SHOW CREATE TABLE replicated_table_for_alter1;
|
||||
|
||||
ALTER TABLE replicated_table_for_alter1 MODIFY SETTING index_granularity = 4096; -- { serverError 469 }
|
||||
|
||||
SHOW CREATE TABLE replicated_table_for_alter1;
|
||||
|
||||
INSERT INTO replicated_table_for_alter2 VALUES (1, '1'), (2, '2');
|
||||
|
||||
SYSTEM SYNC REPLICA replicated_table_for_alter1;
|
||||
|
||||
ALTER TABLE replicated_table_for_alter1 MODIFY SETTING use_minimalistic_part_header_in_zookeeper = 1;
|
||||
|
||||
INSERT INTO replicated_table_for_alter1 VALUES (3, '3'), (4, '4');
|
||||
|
||||
SYSTEM SYNC REPLICA replicated_table_for_alter2;
|
||||
|
||||
SELECT COUNT() FROM replicated_table_for_alter1;
|
||||
SELECT COUNT() FROM replicated_table_for_alter2;
|
||||
|
||||
DETACH TABLE replicated_table_for_alter2;
|
||||
ATTACH TABLE replicated_table_for_alter2;
|
||||
|
||||
DETACH TABLE replicated_table_for_alter1;
|
||||
ATTACH TABLE replicated_table_for_alter1;
|
||||
|
||||
SELECT COUNT() FROM replicated_table_for_alter1;
|
||||
SELECT COUNT() FROM replicated_table_for_alter2;
|
||||
|
||||
ALTER TABLE replicated_table_for_alter2 MODIFY SETTING parts_to_throw_insert = 1, parts_to_delay_insert = 1;
|
||||
INSERT INTO replicated_table_for_alter2 VALUES (3, '1'), (4, '2'); -- { serverError 252 }
|
||||
|
||||
INSERT INTO replicated_table_for_alter1 VALUES (5, '5'), (6, '6');
|
||||
|
||||
SYSTEM SYNC REPLICA replicated_table_for_alter2;
|
||||
|
||||
SELECT COUNT() FROM replicated_table_for_alter1;
|
||||
SELECT COUNT() FROM replicated_table_for_alter2;
|
||||
|
||||
DETACH TABLE replicated_table_for_alter2;
|
||||
ATTACH TABLE replicated_table_for_alter2;
|
||||
|
||||
DETACH TABLE replicated_table_for_alter1;
|
||||
ATTACH TABLE replicated_table_for_alter1;
|
||||
|
||||
SHOW CREATE TABLE replicated_table_for_alter1;
|
||||
SHOW CREATE TABLE replicated_table_for_alter2;
|
||||
|
||||
DROP TABLE IF EXISTS replicated_table_for_alter2;
|
||||
DROP TABLE IF EXISTS replicated_table_for_alter1;
|
Loading…
Reference in New Issue
Block a user