Add ALTER TABLE UNFREEZE command.

This commit is contained in:
Pavel Kovalenko 2021-02-24 17:26:46 +03:00
parent 395526cb80
commit 078dfce038
11 changed files with 93 additions and 5 deletions

View File

@ -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_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_MOVE_PARTITION, "ALTER MOVE PART, MOVE PARTITION, MOVE PART", TABLE, ALTER_TABLE) \
M(ALTER_FETCH_PARTITION, "FETCH PARTITION", 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) \ M(ALTER_TABLE, "", GROUP, ALTER) \
\ \

View File

@ -298,7 +298,8 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
break; break;
} }
case ASTAlterCommand::FREEZE_PARTITION: [[fallthrough]]; 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); required_access.emplace_back(AccessType::ALTER_FREEZE_PARTITION, database, table);
break; break;

View File

@ -271,6 +271,15 @@ void ASTAlterCommand::formatImpl(
<< " " << DB::quote << with_name; << " " << 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) else if (type == ASTAlterCommand::DELETE)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DELETE" << (settings.hilite ? hilite_none : ""); 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 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. */ /** Get the text that identifies this element. */

View File

@ -54,6 +54,7 @@ public:
FETCH_PARTITION, FETCH_PARTITION,
FREEZE_PARTITION, FREEZE_PARTITION,
FREEZE_ALL, FREEZE_ALL,
UNFREEZE,
DELETE, DELETE,
UPDATE, UPDATE,
@ -153,7 +154,9 @@ public:
*/ */
String from; 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; String with_name;

View File

@ -137,6 +137,7 @@ class AlterTableClause : public INode
DROP_INDEX, DROP_INDEX,
DROP_PARTITION, DROP_PARTITION,
FREEZE_PARTITION, FREEZE_PARTITION,
UNFREEZE,
MODIFY, MODIFY,
MOVE_PARTITION_TO_DISK, MOVE_PARTITION_TO_DISK,
MOVE_PARTITION_TO_TABLE, MOVE_PARTITION_TO_TABLE,

View File

@ -63,6 +63,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_fetch_partition("FETCH PARTITION"); ParserKeyword s_fetch_partition("FETCH PARTITION");
ParserKeyword s_replace_partition("REPLACE PARTITION"); ParserKeyword s_replace_partition("REPLACE PARTITION");
ParserKeyword s_freeze("FREEZE"); ParserKeyword s_freeze("FREEZE");
ParserKeyword s_unfreeze("UNFREEZE");
ParserKeyword s_partition("PARTITION"); ParserKeyword s_partition("PARTITION");
ParserKeyword s_first("FIRST"); ParserKeyword s_first("FIRST");
@ -454,6 +455,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->with_name = ast_with_name->as<ASTLiteral &>().value.get<const String &>(); command->with_name = ast_with_name->as<ASTLiteral &>().value.get<const String &>();
} }
} }
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<ASTLiteral &>().value.get<const String &>();
}
else if (s_modify_column.ignore(pos, expected)) else if (s_modify_column.ignore(pos, expected))
{ {
if (s_if_exists.ignore(pos, expected)) if (s_if_exists.ignore(pos, expected))

View File

@ -2920,6 +2920,14 @@ Pipe MergeTreeData::alterPartition(
current_command_results = freezeAll(command.with_name, metadata_snapshot, query_context, lock); current_command_results = freezeAll(command.with_name, metadata_snapshot, query_context, lock);
} }
break; 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) for (auto & command_result : current_command_results)
command_result.command_type = command.typeToString(); command_result.command_type = command.typeToString();
@ -3727,6 +3735,23 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(MatcherFn m
return result; 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 bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const
{ {
const auto settings = getSettings(); const auto settings = getSettings();

View File

@ -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 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: public:
/// Moves partition to specified Disk /// Moves partition to specified Disk

View File

@ -99,6 +99,13 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
command.with_name = command_ast->with_name; command.with_name = command_ast->with_name;
return command; 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 else
return {}; return {};
} }
@ -130,6 +137,8 @@ std::string PartitionCommand::typeToString() const
return "FREEZE ALL"; return "FREEZE ALL";
case PartitionCommand::Type::FREEZE_PARTITION: case PartitionCommand::Type::FREEZE_PARTITION:
return "FREEZE PARTITION"; return "FREEZE PARTITION";
case PartitionCommand::Type::UNFREEZE_PARTITIONS:
return "UNFREEZE";
case PartitionCommand::Type::REPLACE_PARTITION: case PartitionCommand::Type::REPLACE_PARTITION:
return "REPLACE PARTITION"; return "REPLACE PARTITION";
} }

View File

@ -27,6 +27,7 @@ struct PartitionCommand
FETCH_PARTITION, FETCH_PARTITION,
FREEZE_ALL_PARTITIONS, FREEZE_ALL_PARTITIONS,
FREEZE_PARTITION, FREEZE_PARTITION,
UNFREEZE_PARTITIONS,
REPLACE_PARTITION, REPLACE_PARTITION,
}; };
@ -52,7 +53,7 @@ struct PartitionCommand
/// For FETCH PARTITION - path in ZK to the shard, from which to download the partition. /// For FETCH PARTITION - path in ZK to the shard, from which to download the partition.
String from_zookeeper_path; String from_zookeeper_path;
/// For FREEZE PARTITION /// For FREEZE PARTITION and UNFREEZE
String with_name; String with_name;
enum MoveDestinationType enum MoveDestinationType

View File

@ -333,3 +333,26 @@ def test_move_replace_partition_to_another_table(cluster):
for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')):
minio.remove_object(cluster.minio_bucket, obj.object_name) 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