Add some changes

This commit is contained in:
alesapin 2019-08-06 15:52:08 +03:00
parent 86831fe500
commit e12cb8da6d
7 changed files with 177 additions and 37 deletions

View File

@ -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 });
}

View File

@ -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(); });
}
}

View File

@ -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;
};
}

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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;