mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Reading the code of ALTER MOVE PARTITION
This commit is contained in:
parent
93fddc7b46
commit
0dbd885679
@ -1042,6 +1042,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma
|
||||
case PartitionCommand::MoveDestinationType::VOLUME:
|
||||
movePartitionToVolume(command.partition, command.move_destination_name, command.part, context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::MoveDestinationType::TABLE:
|
||||
checkPartitionCanBeDropped(command.partition);
|
||||
String dest_database = command.to_database.empty() ? context.getCurrentDatabase() : command.to_database;
|
||||
@ -1231,7 +1232,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
|
||||
|
||||
auto dest_table_storage = std::dynamic_pointer_cast<StorageMergeTree>(dest_table);
|
||||
if (!dest_table_storage)
|
||||
throw Exception("Table " + getStorageID().getNameForLogs() + " supports attachPartitionFrom only for MergeTree family of table engines."
|
||||
throw Exception("Table " + getStorageID().getNameForLogs() + " supports movePartitionToTable only for MergeTree family of table engines."
|
||||
" Got " + dest_table->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (dest_table_storage->getStoragePolicy() != this->getStoragePolicy())
|
||||
throw Exception("Destination table " + dest_table_storage->getStorageID().getNameForLogs() +
|
||||
@ -1246,13 +1247,13 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
|
||||
DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
|
||||
MutableDataPartsVector dst_parts;
|
||||
|
||||
static const String TMP_PREFIX = "tmp_replace_from_";
|
||||
static const String TMP_PREFIX = "tmp_move_from_";
|
||||
|
||||
for (const DataPartPtr & src_part : src_parts)
|
||||
{
|
||||
if (!dest_table_storage->canReplacePartition(src_part))
|
||||
throw Exception(
|
||||
"Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
|
||||
"Cannot move partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// This will generate unique name in scope of current server process.
|
||||
@ -1263,18 +1264,11 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
|
||||
dst_parts.emplace_back(dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info));
|
||||
}
|
||||
|
||||
/// ATTACH empty part set
|
||||
/// empty part set
|
||||
if (dst_parts.empty())
|
||||
return;
|
||||
|
||||
MergeTreePartInfo drop_range;
|
||||
|
||||
drop_range.partition_id = partition_id;
|
||||
drop_range.min_block = 0;
|
||||
drop_range.max_block = increment.get(); // there will be a "hole" in block numbers
|
||||
drop_range.level = std::numeric_limits<decltype(drop_range.level)>::max();
|
||||
|
||||
/// Atomically add new parts and remove old ones
|
||||
/// Move new parts to the destination table. NOTE It doesn't look atomic.
|
||||
try
|
||||
{
|
||||
{
|
||||
|
@ -3551,9 +3551,11 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part
|
||||
case PartitionCommand::MoveDestinationType::DISK:
|
||||
movePartitionToDisk(command.partition, command.move_destination_name, command.part, query_context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::MoveDestinationType::VOLUME:
|
||||
movePartitionToVolume(command.partition, command.move_destination_name, command.part, query_context);
|
||||
break;
|
||||
|
||||
case PartitionCommand::MoveDestinationType::TABLE:
|
||||
checkPartitionCanBeDropped(command.partition);
|
||||
String dest_database = command.to_database.empty() ? query_context.getCurrentDatabase() : command.to_database;
|
||||
@ -5118,7 +5120,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
|
||||
|
||||
auto dest_table_storage = std::dynamic_pointer_cast<StorageReplicatedMergeTree>(dest_table);
|
||||
if (!dest_table_storage)
|
||||
throw Exception("Table " + getStorageID().getNameForLogs() + " supports attachPartitionFrom only for ReplicatedMergeTree family of table engines."
|
||||
throw Exception("Table " + getStorageID().getNameForLogs() + " supports movePartitionToTable only for ReplicatedMergeTree family of table engines."
|
||||
" Got " + dest_table->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (dest_table_storage->getStoragePolicy() != this->getStoragePolicy())
|
||||
throw Exception("Destination table " + dest_table_storage->getStorageID().getNameForLogs() +
|
||||
@ -5140,9 +5142,11 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
|
||||
|
||||
LOG_DEBUG(log, "Cloning " << src_all_parts.size() << " parts");
|
||||
|
||||
static const String TMP_PREFIX = "tmp_replace_from_";
|
||||
static const String TMP_PREFIX = "tmp_move_from_";
|
||||
auto zookeeper = getZooKeeper();
|
||||
|
||||
/// A range for log entry to remove parts from the source table (myself).
|
||||
|
||||
MergeTreePartInfo drop_range;
|
||||
drop_range.partition_id = partition_id;
|
||||
drop_range.max_block = allocateBlockNumber(partition_id, zookeeper)->getNumber();
|
||||
@ -5157,13 +5161,15 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
|
||||
queue.disableMergesInBlockRange(drop_range_fake_part_name);
|
||||
}
|
||||
|
||||
/// Clone parts into destination table.
|
||||
|
||||
for (size_t i = 0; i < src_all_parts.size(); ++i)
|
||||
{
|
||||
auto & src_part = src_all_parts[i];
|
||||
|
||||
if (!dest_table_storage->canReplacePartition(src_part))
|
||||
throw Exception(
|
||||
"Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
|
||||
"Cannot move partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String hash_hex = src_part->checksums.getTotalChecksumHex();
|
||||
|
Loading…
Reference in New Issue
Block a user