Add uuid.txt to checksums for parts stored on disk

We are breaking backwards compatibility anyway (but agted by a setting)
This commit is contained in:
Nicolae Vartolomei 2020-11-02 18:52:50 +00:00
parent 94293ca3ce
commit 040aba9f85
8 changed files with 15 additions and 18 deletions

View File

@ -190,9 +190,6 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf
if (client_protocol_version < REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION && file_name == IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME)
continue;
if (client_protocol_version < REPLICATION_PROTOCOL_VERSION_WITH_PARTS_UUID && file_name == IMergeTreeDataPart::UUID_FILE_NAME)
continue;
checksums.files[file_name] = {};
}
@ -340,7 +337,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
return part_type == "InMemory" ? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, std::move(reservation), in)
: downloadPartToDisk(part_name, part_uuid, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in);
: downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in);
}
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
@ -378,7 +375,6 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
const String & part_name,
const UUID & part_uuid,
const String & replica_path,
bool to_detached,
const String & tmp_prefix_,
@ -445,8 +441,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
if (file_name != "checksums.txt" &&
file_name != "columns.txt" &&
file_name != IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME &&
file_name != IMergeTreeDataPart::UUID_FILE_NAME)
file_name != IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME)
checksums.addFile(file_name, file_size, expected_hash);
if (sync)
@ -457,7 +452,6 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk, 0);
MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, volume, part_relative_path);
new_data_part->uuid = part_uuid;
new_data_part->is_temp = true;
new_data_part->modification_time = time(nullptr);
new_data_part->loadColumnsChecksumsIndexes(true, false);

View File

@ -71,7 +71,6 @@ public:
private:
MergeTreeData::MutableDataPartPtr downloadPartToDisk(
const String & part_name,
const UUID & part_uuid,
const String & replica_path,
bool to_detached,
const String & tmp_prefix_,

View File

@ -487,10 +487,6 @@ NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const
if (volume->getDisk()->exists(default_codec_path))
result.emplace(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
String uuid_path = getFullRelativePath() + UUID_FILE_NAME;
if (volume->getDisk()->exists(uuid_path))
result.emplace(UUID_FILE_NAME);
return result;
}
@ -913,7 +909,6 @@ void IMergeTreeDataPart::remove() const
for (const auto & file : {"checksums.txt", "columns.txt"})
volume->getDisk()->remove(to + "/" + file);
volume->getDisk()->removeIfExists(to + "/" + UUID_FILE_NAME);
volume->getDisk()->removeIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME);
volume->getDisk()->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME);

View File

@ -732,6 +732,7 @@ protected:
friend class MergeTreeDataMergerMutator;
friend struct ReplicatedMergeTreeTableMetadata;
friend class StorageReplicatedMergeTree;
friend class MergeTreeDataWriter;
bool require_part_metadata;

View File

@ -1824,7 +1824,10 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart(
if (new_data_part->uuid != UUIDHelpers::Nil)
{
auto out = disk->writeFile(new_data_part->getFullRelativePath() + IMergeTreeDataPart::UUID_FILE_NAME, 4096);
writeUUIDText(new_data_part->uuid, *out);
HashingWriteBuffer out_hashing(*out);
writeUUIDText(new_data_part->uuid, out_hashing);
new_data_part->checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count();
new_data_part->checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash();
}
if (need_remove_expired_values)

View File

@ -247,7 +247,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
createVolumeFromReservation(reservation, volume),
TMP_PREFIX + part_name);
new_data_part->uuid = UUIDHelpers::generateV4();
if (data.storage_settings.get()->assign_part_uuids)
new_data_part->uuid = UUIDHelpers::generateV4();
new_data_part->setColumns(columns);
new_data_part->partition = std::move(partition);
new_data_part->minmax_idx = std::move(minmax_idx);

View File

@ -136,7 +136,10 @@ void MergedBlockOutputStream::finalizePartOnDisk(
if (new_part->uuid != UUIDHelpers::Nil)
{
auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::UUID_FILE_NAME, 4096);
writeUUIDText(new_part->uuid, *out);
HashingWriteBuffer out_hashing(*out);
writeUUIDText(new_part->uuid, out_hashing);
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count();
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash();
out->finalize();
if (sync)
out->sync();

View File

@ -24,7 +24,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const
/// Conditionally bump format_version only when uuid has been assigned.
/// If some other feature requires bumping format_version to >= 5 then this code becomes no-op.
if (new_part_uuid != UUIDHelpers::Nil)
format_version = std::min(format_version, static_cast<UInt8>(5));
format_version = std::max(format_version, static_cast<UInt8>(5));
out << "format version: " << format_version << "\n"
<< "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n"