ALTER TABLE UNFREEZE rework.

This commit is contained in:
Pavel Kovalenko 2021-03-03 00:10:09 +03:00
parent a3c0c35dff
commit 483e63ca45
3 changed files with 35 additions and 15 deletions

View File

@ -299,7 +299,8 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
}
case ASTAlterCommand::FREEZE_PARTITION: [[fallthrough]];
case ASTAlterCommand::FREEZE_ALL: [[fallthrough]];
case ASTAlterCommand::UNFREEZE:
case ASTAlterCommand::UNFREEZE_PARTITION: [[fallthrough]];
case ASTAlterCommand::UNFREEZE_ALL:
{
required_access.emplace_back(AccessType::ALTER_FREEZE_PARTITION, database, table);
break;

View File

@ -3678,17 +3678,30 @@ MergeTreeData::MatcherFn MergeTreeData::getPartitionMatcher(const ASTPtr & parti
};
}
PartitionCommandsResultInfo MergeTreeData::freezePartition(const ASTPtr & partition_ast, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context, TableLockHolder &)
PartitionCommandsResultInfo MergeTreeData::freezePartition(
const ASTPtr & partition_ast,
const StorageMetadataPtr & metadata_snapshot,
const String & with_name,
const Context & context,
TableLockHolder &)
{
return freezePartitionsByMatcher(getPartitionMatcher(partition_ast, context), metadata_snapshot, with_name, context);
}
PartitionCommandsResultInfo MergeTreeData::freezeAll(const String & with_name, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableLockHolder &)
PartitionCommandsResultInfo MergeTreeData::freezeAll(
const String & with_name,
const StorageMetadataPtr & metadata_snapshot,
const Context & context,
TableLockHolder &)
{
return freezePartitionsByMatcher([] (const String &) { return true; }, metadata_snapshot, with_name, context);
}
PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context)
PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
MatcherFn matcher,
const StorageMetadataPtr & metadata_snapshot,
const String & with_name,
const Context & context)
{
String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString();
String default_shadow_path = clickhouse_path + "shadow/";
@ -3758,10 +3771,12 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll(
return unfreezePartitionsByMatcher([] (const String &) { return true; }, backup_name, context);
}
PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, const Context & context)
PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, const Context &)
{
auto backup_path = std::filesystem::path("shadow") / escapeForFileName(backup_name) / relative_data_path;
LOG_DEBUG(log, "Unfreezing parts by path {}", backup_path.generic_string());
PartitionCommandsResultInfo result;
for (const auto & disk : getStoragePolicy()->getDisks())
@ -3782,17 +3797,19 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn
if (!matcher(partition_id))
continue;
disk->removeRecursive(it->path());
auto path = it->path();
disk->removeRecursive(path);
result.push_back(PartitionCommandResultInfo{
.partition_id = partition_id,
.part_name = partition_directory,
.backup_path = disk->getPath() + backup_path.generic_string(),
.part_backup_path = disk->getPath() + it->path(),
.part_backup_path = disk->getPath() + path,
.backup_name = backup_name,
});
LOG_DEBUG(log, "Unfreezed part by path {}", disk->getPath() + it->path());
LOG_DEBUG(log, "Unfreezed part by path {}", disk->getPath() + path);
}
}

View File

@ -342,17 +342,19 @@ def test_freeze_unfreeze(cluster):
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("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-04', 4096)))
node.query("ALTER TABLE s3_test FREEZE WITH NAME 'backup2'")
node.query("TRUNCATE TABLE s3_test")
assert len(
list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE
list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
node.query("ALTER TABLE s3_test UNFREEZE 'backup1'")
# Unfreeze single partition from backup1.
node.query("ALTER TABLE s3_test UNFREEZE PARTITION '2020-01-03' WITH NAME 'backup1'")
# Unfreeze all partitions from backup2.
node.query("ALTER TABLE s3_test UNFREEZE WITH NAME 'backup2'")
# Data should be removed from S3.
assert len(
list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD