diff --git a/src/Access/AccessType.h b/src/Access/AccessType.h index 5a84aa66739..db3798fbf63 100644 --- a/src/Access/AccessType.h +++ b/src/Access/AccessType.h @@ -63,7 +63,7 @@ enum class AccessType M(ALTER_SETTINGS, "ALTER SETTING, ALTER MODIFY SETTING, MODIFY SETTING", TABLE, ALTER_TABLE) /* allows to execute ALTER MODIFY SETTING */\ M(ALTER_MOVE_PARTITION, "ALTER MOVE PART, MOVE PARTITION, MOVE PART", TABLE, ALTER_TABLE) \ M(ALTER_FETCH_PARTITION, "FETCH PARTITION", TABLE, ALTER_TABLE) \ - M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION", TABLE, ALTER_TABLE) \ + M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ \ M(ALTER_TABLE, "", GROUP, ALTER) \ \ diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index bf624507574..c45ec945321 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -298,7 +298,8 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS break; } case ASTAlterCommand::FREEZE_PARTITION: [[fallthrough]]; - case ASTAlterCommand::FREEZE_ALL: + case ASTAlterCommand::FREEZE_ALL: [[fallthrough]]; + case ASTAlterCommand::UNFREEZE: { required_access.emplace_back(AccessType::ALTER_FREEZE_PARTITION, database, table); break; diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index f24b26d5b54..9b1eea755b9 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -271,6 +271,15 @@ void ASTAlterCommand::formatImpl( << " " << DB::quote << with_name; } } + else if (type == ASTAlterCommand::UNFREEZE) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "UNFREEZE"; + + if (!with_name.empty()) + { + settings.ostr << " " << DB::quote << with_name; + } + } else if (type == ASTAlterCommand::DELETE) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DELETE" << (settings.hilite ? hilite_none : ""); @@ -368,7 +377,8 @@ bool ASTAlterQuery::isSettingsAlter() const bool ASTAlterQuery::isFreezeAlter() const { - return isOneCommandTypeOnly(ASTAlterCommand::FREEZE_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::FREEZE_ALL); + return isOneCommandTypeOnly(ASTAlterCommand::FREEZE_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::FREEZE_ALL) + || isOneCommandTypeOnly(ASTAlterCommand::UNFREEZE); } /** Get the text that identifies this element. */ diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 4cc01aa889e..7934a9eb790 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -54,6 +54,7 @@ public: FETCH_PARTITION, FREEZE_PARTITION, FREEZE_ALL, + UNFREEZE, DELETE, UPDATE, @@ -153,7 +154,9 @@ public: */ String from; - /** For FREEZE PARTITION - place local backup to directory with specified name. + /** + * For FREEZE PARTITION - place local backup to directory with specified name. + * For UNFREEZE - delete local backup at directory with specified name. */ String with_name; diff --git a/src/Parsers/New/AST/AlterTableQuery.h b/src/Parsers/New/AST/AlterTableQuery.h index a1e5becedaf..98f07bb0d71 100644 --- a/src/Parsers/New/AST/AlterTableQuery.h +++ b/src/Parsers/New/AST/AlterTableQuery.h @@ -137,6 +137,7 @@ class AlterTableClause : public INode DROP_INDEX, DROP_PARTITION, FREEZE_PARTITION, + UNFREEZE, MODIFY, MOVE_PARTITION_TO_DISK, MOVE_PARTITION_TO_TABLE, diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 5d20e27e486..2e907e38b44 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -63,6 +63,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_fetch_partition("FETCH PARTITION"); ParserKeyword s_replace_partition("REPLACE PARTITION"); ParserKeyword s_freeze("FREEZE"); + ParserKeyword s_unfreeze("UNFREEZE"); ParserKeyword s_partition("PARTITION"); ParserKeyword s_first("FIRST"); @@ -454,6 +455,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->with_name = ast_with_name->as().value.get(); } } + else if (s_unfreeze.ignore(pos, expected)) + { + command->type = ASTAlterCommand::UNFREEZE; + + ASTPtr ast_with_name; + if (!parser_string_literal.parse(pos, ast_with_name, expected)) + return false; + + command->with_name = ast_with_name->as().value.get(); + } else if (s_modify_column.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b09f068f509..da00e4361ce 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2920,6 +2920,14 @@ Pipe MergeTreeData::alterPartition( current_command_results = freezeAll(command.with_name, metadata_snapshot, query_context, lock); } break; + + case PartitionCommand::UNFREEZE_PARTITIONS: + { + auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); + current_command_results = unfreeze(command.with_name, query_context, lock); + } + + break; } for (auto & command_result : current_command_results) command_result.command_type = command.typeToString(); @@ -3727,6 +3735,23 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(MatcherFn m return result; } +PartitionCommandsResultInfo MergeTreeData::unfreeze( + const String & backup_name, + const Context & context, + TableLockHolder & ) const +{ + String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString(); + String backup_path = clickhouse_path + "shadow/" + backup_name + "/" + relative_data_path; + + PartitionCommandsResultInfo result; + + for (const auto & disk : getStoragePolicy()->getDisks()) + if (disk->exists(backup_path)) + disk->removeRecursive(backup_path); + + return result; +} + bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const { const auto settings = getSettings(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2aefa66ac58..a5b56bcd8d4 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -553,6 +553,10 @@ public: */ PartitionCommandsResultInfo freezePartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context, TableLockHolder & table_lock_holder); + PartitionCommandsResultInfo unfreeze( + const String & backup_name, + const Context & context, + TableLockHolder & table_lock_holder) const; public: /// Moves partition to specified Disk diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 76c2af17256..79e30067b77 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -99,6 +99,13 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * command.with_name = command_ast->with_name; return command; } + else if (command_ast->type == ASTAlterCommand::UNFREEZE) + { + PartitionCommand command; + command.type = PartitionCommand::UNFREEZE_PARTITIONS; + command.with_name = command_ast->with_name; + return command; + } else return {}; } @@ -130,6 +137,8 @@ std::string PartitionCommand::typeToString() const return "FREEZE ALL"; case PartitionCommand::Type::FREEZE_PARTITION: return "FREEZE PARTITION"; + case PartitionCommand::Type::UNFREEZE_PARTITIONS: + return "UNFREEZE"; case PartitionCommand::Type::REPLACE_PARTITION: return "REPLACE PARTITION"; } diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index e4f70305dbd..1679943d756 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -27,6 +27,7 @@ struct PartitionCommand FETCH_PARTITION, FREEZE_ALL_PARTITIONS, FREEZE_PARTITION, + UNFREEZE_PARTITIONS, REPLACE_PARTITION, }; @@ -52,7 +53,7 @@ struct PartitionCommand /// For FETCH PARTITION - path in ZK to the shard, from which to download the partition. String from_zookeeper_path; - /// For FREEZE PARTITION + /// For FREEZE PARTITION and UNFREEZE String with_name; enum MoveDestinationType diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 45b3c3c65f0..6acc5ea6a2e 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -333,3 +333,26 @@ def test_move_replace_partition_to_another_table(cluster): for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): minio.remove_object(cluster.minio_bucket, obj.object_name) + + +def test_freeze_unfreeze(cluster): + create_table(cluster, "s3_test") + + node = cluster.instances["node"] + minio = cluster.minio_client + + node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-03', 4096))) + assert len( + list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + + node.query("ALTER TABLE s3_test FREEZE WITH NAME 'backup1'") + assert len( + list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + + node.query("TRUNCATE TABLE s3_test") + assert len( + list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + + node.query("ALTER TABLE s3_test UNFREEZE 'backup1'") + assert len( + list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD