Use proper fmt:: like Exception ctor in DataPartsExchange

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2022-02-01 13:00:23 +03:00
parent cf93732de9
commit 743096a883

View File

@ -222,7 +222,7 @@ void Service::sendPartFromMemory(
auto projection_sample_block = metadata_snapshot->projections.get(name).sample_block; auto projection_sample_block = metadata_snapshot->projections.get(name).sample_block;
auto part_in_memory = asInMemoryPart(projection); auto part_in_memory = asInMemoryPart(projection);
if (!part_in_memory) if (!part_in_memory)
throw Exception("Projection " + name + " of part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR); throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection {} of part {} is not stored in memory", name, part->name);
writeStringBinary(name, out); writeStringBinary(name, out);
projection->checksums.write(out); projection->checksums.write(out);
@ -232,7 +232,7 @@ void Service::sendPartFromMemory(
auto part_in_memory = asInMemoryPart(part); auto part_in_memory = asInMemoryPart(part);
if (!part_in_memory) if (!part_in_memory)
throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR); throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} is not stored in memory", part->name);
NativeWriter block_out(out, 0, metadata_snapshot->getSampleBlock()); NativeWriter block_out(out, 0, metadata_snapshot->getSampleBlock());
part->checksums.write(out); part->checksums.write(out);
@ -300,7 +300,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
if (hashing_out.count() != size) if (hashing_out.count() != size)
throw Exception("Unexpected size of file " + path, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected size of file {}", path);
writePODBinary(hashing_out.getHash(), out); writePODBinary(hashing_out.getHash(), out);
@ -323,7 +323,7 @@ void Service::sendPartFromDiskRemoteMeta(const MergeTreeData::DataPartPtr & part
auto disk = part->volume->getDisk(); auto disk = part->volume->getDisk();
if (!disk->supportZeroCopyReplication()) if (!disk->supportZeroCopyReplication())
throw Exception(fmt::format("disk {} doesn't support zero-copy replication", disk->getName()), ErrorCodes::LOGICAL_ERROR); throw Exception(ErrorCodes::LOGICAL_ERROR, "disk {} doesn't support zero-copy replication", disk->getName());
part->storage.lockSharedData(*part); part->storage.lockSharedData(*part);
@ -340,9 +340,9 @@ void Service::sendPartFromDiskRemoteMeta(const MergeTreeData::DataPartPtr & part
fs::path metadata(metadata_file); fs::path metadata(metadata_file);
if (!fs::exists(metadata)) if (!fs::exists(metadata))
throw Exception("Remote metadata '" + file_name + "' is not exists", ErrorCodes::CORRUPTED_DATA); throw Exception(ErrorCodes::CORRUPTED_DATA, "Remote metadata '{}' is not exists", file_name);
if (!fs::is_regular_file(metadata)) if (!fs::is_regular_file(metadata))
throw Exception("Remote metadata '" + file_name + "' is not a file", ErrorCodes::CORRUPTED_DATA); throw Exception(ErrorCodes::CORRUPTED_DATA, "Remote metadata '{}' is not a file", file_name);
UInt64 file_size = fs::file_size(metadata); UInt64 file_size = fs::file_size(metadata);
writeStringBinary(it.first, out); writeStringBinary(it.first, out);
@ -355,7 +355,7 @@ void Service::sendPartFromDiskRemoteMeta(const MergeTreeData::DataPartPtr & part
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
if (hashing_out.count() != file_size) if (hashing_out.count() != file_size)
throw Exception("Unexpected size of file " + metadata_file, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected size of file {}", metadata_file);
writePODBinary(hashing_out.getHash(), out); writePODBinary(hashing_out.getHash(), out);
} }
@ -370,7 +370,7 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name)
if (part) if (part)
return part; return part;
throw Exception("No part " + name + " in table", ErrorCodes::NO_SUCH_DATA_PART); throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in table", name);
} }
MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
@ -511,9 +511,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
if (!try_zero_copy) if (!try_zero_copy)
throw Exception("Got unexpected 'remote_fs_metadata' cookie", ErrorCodes::LOGICAL_ERROR); throw Exception("Got unexpected 'remote_fs_metadata' cookie", ErrorCodes::LOGICAL_ERROR);
if (std::find(capability.begin(), capability.end(), remote_fs_metadata) == capability.end()) if (std::find(capability.begin(), capability.end(), remote_fs_metadata) == capability.end())
throw Exception(fmt::format("Got 'remote_fs_metadata' cookie {}, expect one from {}", remote_fs_metadata, fmt::join(capability, ", ")), ErrorCodes::LOGICAL_ERROR); throw Exception(ErrorCodes::LOGICAL_ERROR, "Got 'remote_fs_metadata' cookie {}, expect one from {}", remote_fs_metadata, fmt::join(capability, ", "));
if (server_protocol_version < REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY) if (server_protocol_version < REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY)
throw Exception(fmt::format("Got 'remote_fs_metadata' cookie with old protocol version {}", server_protocol_version), ErrorCodes::LOGICAL_ERROR); throw Exception(ErrorCodes::LOGICAL_ERROR, "Got 'remote_fs_metadata' cookie with old protocol version {}", server_protocol_version);
if (part_type == "InMemory") if (part_type == "InMemory")
throw Exception("Got 'remote_fs_metadata' cookie for in-memory part", ErrorCodes::INCORRECT_PART_TYPE); throw Exception("Got 'remote_fs_metadata' cookie for in-memory part", ErrorCodes::INCORRECT_PART_TYPE);
@ -649,9 +649,10 @@ void Fetcher::downloadBaseOrProjectionPartToDisk(
/// Otherwise malicious ClickHouse replica may force us to write to arbitrary path. /// Otherwise malicious ClickHouse replica may force us to write to arbitrary path.
String absolute_file_path = fs::weakly_canonical(fs::path(part_download_path) / file_name); String absolute_file_path = fs::weakly_canonical(fs::path(part_download_path) / file_name);
if (!startsWith(absolute_file_path, fs::weakly_canonical(part_download_path).string())) if (!startsWith(absolute_file_path, fs::weakly_canonical(part_download_path).string()))
throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + part_download_path + ")." throw Exception(ErrorCodes::INSECURE_PATH,
" This may happen if we are trying to download part from malicious replica or logical error.", "File path ({}) doesn't appear to be inside part path ({}). "
ErrorCodes::INSECURE_PATH); "This may happen if we are trying to download part from malicious replica or logical error.",
absolute_file_path, part_download_path);
auto file_out = disk->writeFile(fs::path(part_download_path) / file_name); auto file_out = disk->writeFile(fs::path(part_download_path) / file_name);
HashingWriteBuffer hashing_out(*file_out); HashingWriteBuffer hashing_out(*file_out);
@ -670,8 +671,10 @@ void Fetcher::downloadBaseOrProjectionPartToDisk(
readPODBinary(expected_hash, in); readPODBinary(expected_hash, in);
if (expected_hash != hashing_out.getHash()) if (expected_hash != hashing_out.getHash())
throw Exception("Checksum mismatch for file " + fullPath(disk, (fs::path(part_download_path) / file_name).string()) + " transferred from " + replica_path, throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH,
ErrorCodes::CHECKSUM_DOESNT_MATCH); "Checksum mismatch for file {} transferred from {}",
fullPath(disk, (fs::path(part_download_path) / file_name).string()),
replica_path);
if (file_name != "checksums.txt" && if (file_name != "checksums.txt" &&
file_name != "columns.txt" && file_name != "columns.txt" &&
@ -762,7 +765,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDiskRemoteMeta(
if (!disk->supportZeroCopyReplication() || !disk->checkUniqueId(part_id)) if (!disk->supportZeroCopyReplication() || !disk->checkUniqueId(part_id))
{ {
throw Exception(fmt::format("Part {} unique id {} doesn't exist on {}.", part_name, part_id, disk->getName()), ErrorCodes::ZERO_COPY_REPLICATION_ERROR); throw Exception(ErrorCodes::ZERO_COPY_REPLICATION_ERROR, "Part {} unique id {} doesn't exist on {}.", part_name, part_id, disk->getName());
} }
LOG_DEBUG(log, "Downloading Part {} unique id {} metadata onto disk {}.", LOG_DEBUG(log, "Downloading Part {} unique id {} metadata onto disk {}.",
part_name, part_id, disk->getName()); part_name, part_id, disk->getName());
@ -774,7 +777,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDiskRemoteMeta(
String part_download_path = fs::path(data.getRelativeDataPath()) / part_relative_path / ""; String part_download_path = fs::path(data.getRelativeDataPath()) / part_relative_path / "";
if (disk->exists(part_download_path)) if (disk->exists(part_download_path))
throw Exception("Directory " + fullPath(disk, part_download_path) + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS); throw Exception(ErrorCodes::DIRECTORY_ALREADY_EXISTS, "Directory {} already exists.", fullPath(disk, part_download_path));
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedFetch}; CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedFetch};
@ -817,8 +820,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDiskRemoteMeta(
if (expected_hash != hashing_out.getHash()) if (expected_hash != hashing_out.getHash())
{ {
throw Exception("Checksum mismatch for file " + metadata_file + " transferred from " + replica_path, throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH,
ErrorCodes::CHECKSUM_DOESNT_MATCH); "Checksum mismatch for file {} transferred from {}",
metadata_file, replica_path);
} }
} }
} }