From daaa5cc382947ef47c96840643d1bbc1adacf74e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Jul 2020 17:05:30 +0300 Subject: [PATCH] Add setting allow_non_metadata_alters --- programs/client/TestHint.h | 8 +- src/Core/Settings.h | 3 +- src/DataTypes/DataTypeEnum.cpp | 6 +- src/DataTypes/DataTypeEnum.h | 5 +- src/Storages/AlterCommands.cpp | 55 ++++--------- src/Storages/MergeTree/MergeTreeData.cpp | 82 +++++-------------- src/Storages/MutationCommands.h | 2 +- .../01344_alter_enum_partition_key.sql | 2 +- ...46_alter_enum_partition_key_replicated.sql | 3 +- 9 files changed, 57 insertions(+), 109 deletions(-) diff --git a/programs/client/TestHint.h b/programs/client/TestHint.h index 81937435b2e..641c3e0ccf0 100644 --- a/programs/client/TestHint.h +++ b/programs/client/TestHint.h @@ -22,8 +22,9 @@ namespace ErrorCodes class TestHint { public: - TestHint(bool enabled_, const String & query) - : enabled(enabled_) + TestHint(bool enabled_, const String & query_) + : enabled(enabled_) + , query(query_) { if (!enabled_) return; @@ -71,7 +72,7 @@ public: if (lostExpectedError(actual_server_error, actual_client_error)) { - std::cerr << "Success when error expected. It expects server error " + std::cerr << "Success when error expected in query: " << query << "It expects server error " << server_error << ", client error " << client_error << "." << std::endl; got_exception = true; last_exception = std::make_unique("Success when error expected", ErrorCodes::UNEXPECTED_ERROR_CODE); /// return error to OS @@ -86,6 +87,7 @@ public: private: bool enabled = false; + const String & query; int server_error = 0; int client_error = 0; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3cd72b7825d..0655b6b2ebb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -469,7 +469,8 @@ struct Settings : public SettingsCollection M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ \ M(SettingBool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ - M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) + M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ + M(SettingBool, allow_non_metadata_alters, true, "Allow to execute alters which not only tables metadata, but also data on disk", 0) \ #define LIST_OF_SETTINGS(M) \ COMMON_SETTINGS(M) \ diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index 8fcd19bce7d..9ad6a9cb690 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -354,8 +354,12 @@ bool DataTypeEnum::contains(const IDataType & rhs) const auto check = [&](const auto & value) { auto it = name_to_value_map.find(value.first); + /// If we don't have this name, than we have to be sure, + /// that this value exists in enum if (it == name_to_value_map.end()) - return false; + return value_to_name_map.count(value.second) > 0; + + /// If we have this name, than it should have the same value return it->value.second == value.second; }; diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index dd41ee3fcc7..80b41692cdd 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -120,7 +120,10 @@ public: bool textCanContainOnlyValidUTF8() const override; size_t getSizeOfValueInMemory() const override { return sizeof(FieldType); } - /// Check current Enum type extends another Enum type (contains all fields with same values). + /// Check current Enum type extends another Enum type (contains all the same values and doesn't override name's with other values) + /// Example: + /// Enum('a' = 1, 'b' = 2) -> Enum('c' = 1, 'b' = 2, 'd' = 3) OK + /// Enum('a' = 1, 'b' = 2) -> Enum('a' = 2, 'b' = 1) NOT OK bool contains(const IDataType & rhs) const; }; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index ca2a0aed108..8e0cd40f605 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -501,32 +501,6 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } -bool AlterCommand::isModifyingData(const StorageInMemoryMetadata & metadata) const -{ - /// Possible change data representation on disk - if (type == MODIFY_COLUMN) - { - if (data_type == nullptr) - return false; - - /// It is allowed to ALTER data type to the same type as before. - for (const auto & column : metadata.columns.getAllPhysical()) - if (column.name == column_name) - return !column.type->equals(*data_type); - - return true; - } - - return type == ADD_COLUMN /// We need to change columns.txt in each part for MergeTree - || type == DROP_COLUMN /// We need to change columns.txt in each part for MergeTree - || type == DROP_INDEX; /// We need to remove file from filesystem for MergeTree -} - -bool AlterCommand::isSettingsAlter() const -{ - return type == MODIFY_SETTING; -} - namespace { @@ -538,11 +512,21 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to) if (from->equals(*to)) return true; + if (const auto * from_enum8 = typeid_cast(from)) + { + if (const auto * to_enum8 = typeid_cast(to)) + return to_enum8->contains(*from_enum8); + } + + if (const auto * from_enum16 = typeid_cast(from)) + { + if (const auto * to_enum16 = typeid_cast(to)) + return to_enum16->contains(*from_enum16); + } + static const std::unordered_multimap ALLOWED_CONVERSIONS = { - { typeid(DataTypeEnum8), typeid(DataTypeEnum8) }, { typeid(DataTypeEnum8), typeid(DataTypeInt8) }, - { typeid(DataTypeEnum16), typeid(DataTypeEnum16) }, { typeid(DataTypeEnum16), typeid(DataTypeInt16) }, { typeid(DataTypeDateTime), typeid(DataTypeUInt32) }, { typeid(DataTypeUInt32), typeid(DataTypeDateTime) }, @@ -583,6 +567,10 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to) } +bool AlterCommand::isSettingsAlter() const +{ + return type == MODIFY_SETTING; +} bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metadata) const { @@ -1009,17 +997,6 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context); } -bool AlterCommands::isModifyingData(const StorageInMemoryMetadata & metadata) const -{ - for (const auto & param : *this) - { - if (param.isModifyingData(metadata)) - return true; - } - - return false; -} - bool AlterCommands::isSettingsAlter() const { return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter(); }); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0501f127b70..73dfc490ae9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1272,60 +1272,7 @@ void MergeTreeData::dropIfEmpty() } } -namespace -{ - -/// If true, then in order to ALTER the type of the column from the type from to the type to -/// we don't need to rewrite the data, we only need to update metadata and columns.txt in part directories. -/// The function works for Arrays and Nullables of the same structure. -bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to) -{ - if (from->getName() == to->getName()) - return true; - - static const std::unordered_multimap ALLOWED_CONVERSIONS = - { - { typeid(DataTypeEnum8), typeid(DataTypeEnum8) }, - { typeid(DataTypeEnum8), typeid(DataTypeInt8) }, - { typeid(DataTypeEnum16), typeid(DataTypeEnum16) }, - { typeid(DataTypeEnum16), typeid(DataTypeInt16) }, - { typeid(DataTypeDateTime), typeid(DataTypeUInt32) }, - { typeid(DataTypeUInt32), typeid(DataTypeDateTime) }, - { typeid(DataTypeDate), typeid(DataTypeUInt16) }, - { typeid(DataTypeUInt16), typeid(DataTypeDate) }, - }; - - while (true) - { - auto it_range = ALLOWED_CONVERSIONS.equal_range(typeid(*from)); - for (auto it = it_range.first; it != it_range.second; ++it) - { - if (it->second == typeid(*to)) - return true; - } - - const auto * arr_from = typeid_cast(from); - const auto * arr_to = typeid_cast(to); - if (arr_from && arr_to) - { - from = arr_from->getNestedType().get(); - to = arr_to->getNestedType().get(); - continue; - } - - const auto * nullable_from = typeid_cast(from); - const auto * nullable_to = typeid_cast(to); - if (nullable_from && nullable_to) - { - from = nullable_from->getNestedType().get(); - to = nullable_to->getNestedType().get(); - continue; - } - - return false; - } -} - +namespace { /// Conversion that is allowed for partition key. /// Partition key should be serialized in the same way after conversion. /// NOTE: The list is not complete. @@ -1361,11 +1308,20 @@ bool isSafeForPartitionKeyConversion(const IDataType * from, const IDataType * t } -void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings &) const +void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const { /// Check that needed transformations can be applied to the list of columns without considering type conversions. StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); + + if (!settings.allow_non_metadata_alters) + { + + auto mutation_commands = commands.getMutationCommands(new_metadata, settings.materialize_ttl_after_modify, global_context); + + if (!mutation_commands.empty()) + throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, "The following alter commands: '{}' will modify data on disk, but setting `allow_non_metadata_alters` is disabled", queryToString(mutation_commands.ast())); + } commands.apply(new_metadata, global_context); /// Set of columns that shouldn't be altered. @@ -1467,8 +1423,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S } dropped_columns.emplace(command.column_name); } - else if (command.isModifyingData(getInMemoryMetadata())) + else if (command.isRequireMutationStage(getInMemoryMetadata())) { + /// This alter will override data on disk. Let's check that it doesn't + /// modify immutable column. if (columns_alter_type_forbidden.count(command.column_name)) throw Exception("ALTER of key column " + backQuoteIfNeed(command.column_name) + " is forbidden", ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); @@ -1478,7 +1436,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S if (command.type == AlterCommand::MODIFY_COLUMN) { auto it = old_types.find(command.column_name); - if (it == old_types.end() || !isSafeForPartitionKeyConversion(it->second, command.data_type.get())) + + assert(it != old_types.end()); + if (!isSafeForPartitionKeyConversion(it->second, command.data_type.get())) throw Exception("ALTER of partition key column " + backQuoteIfNeed(command.column_name) + " from type " + it->second->getName() + " to type " + command.data_type->getName() + " is not safe because it can change the representation of partition key", @@ -1491,10 +1451,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S if (command.type == AlterCommand::MODIFY_COLUMN) { auto it = old_types.find(command.column_name); - if (it == old_types.end() || !isMetadataOnlyConversion(it->second, command.data_type.get())) - throw Exception("ALTER of key column " + backQuoteIfNeed(command.column_name) + " from type " - + it->second->getName() + " to type " + command.data_type->getName() + " must be metadata-only", - ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + assert(it != old_types.end()); + throw Exception("ALTER of key column " + backQuoteIfNeed(command.column_name) + " from type " + + it->second->getName() + " to type " + command.data_type->getName() + " must be metadata-only", + ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); } } } diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 6fa4f7fb641..50541d5c58a 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -28,7 +28,7 @@ struct MutationCommand DELETE, UPDATE, MATERIALIZE_INDEX, - READ_COLUMN, + READ_COLUMN, /// Read column and apply conversions (MODIFY COLUMN alter query). DROP_COLUMN, DROP_INDEX, MATERIALIZE_TTL, diff --git a/tests/queries/0_stateless/01344_alter_enum_partition_key.sql b/tests/queries/0_stateless/01344_alter_enum_partition_key.sql index be6b48918ee..ce9d544f311 100644 --- a/tests/queries/0_stateless/01344_alter_enum_partition_key.sql +++ b/tests/queries/0_stateless/01344_alter_enum_partition_key.sql @@ -12,7 +12,7 @@ SELECT * FROM test ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2); -- { serverError 524 } -ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3); -- { serverError 524 } +ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3); ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 4); -- { serverError 524 } ALTER TABLE test MODIFY COLUMN x Int8; diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated.sql index 35faf6f43ec..1a6c30d9571 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated.sql @@ -21,7 +21,8 @@ SELECT name, partition, partition_id FROM system.parts WHERE database = currentD SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2); -- { serverError 524 } -ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3); -- { serverError 524 } +ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3); + ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 4); -- { serverError 524 } ALTER TABLE test MODIFY COLUMN x Int8;