mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Add ALTER TABLE UNFREEZE command.
This commit is contained in:
parent
395526cb80
commit
078dfce038
@ -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) \
|
||||||
\
|
\
|
||||||
|
@ -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;
|
||||||
|
@ -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. */
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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,
|
||||||
|
@ -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))
|
||||||
|
@ -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();
|
||||||
|
@ -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
|
||||||
|
@ -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";
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
Loading…
Reference in New Issue
Block a user