Use proper fmt-like logging

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2022-02-01 12:52:02 +03:00
parent 3b3635c6d5
commit 5be76bc969
5 changed files with 11 additions and 9 deletions

View File

@ -1428,7 +1428,7 @@ void IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & di
if (disk->exists(fs::path(path_to_clone) / relative_path))
{
LOG_WARNING(storage.log, "Path " + fullPath(disk, path_to_clone + relative_path) + " already exists. Will remove it and clone again.");
LOG_WARNING(storage.log, "Path {} already exists. Will remove it and clone again.", fullPath(disk, path_to_clone + relative_path));
disk->removeRecursive(fs::path(path_to_clone) / relative_path / "");
}
disk->createDirectories(path_to_clone);

View File

@ -1677,7 +1677,7 @@ size_t MergeTreeData::clearOldWriteAheadLogs()
auto min_max_block_number = MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(it->name());
if (min_max_block_number && is_range_on_disk(min_max_block_number->first, min_max_block_number->second))
{
LOG_DEBUG(log, "Removing from filesystem the outdated WAL file " + it->name());
LOG_DEBUG(log, "Removing from filesystem the outdated WAL file {}", it->name());
disk_ptr->removeFile(relative_data_path + it->name());
++cleared_count;
}

View File

@ -211,7 +211,7 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt
String relative_path = part->relative_path;
if (disk->exists(path_to_clone + relative_path))
{
LOG_WARNING(log, "Path " + fullPath(disk, path_to_clone + relative_path) + " already exists. Will remove it and clone again.");
LOG_WARNING(log, "Path {} already exists. Will remove it and clone again.", fullPath(disk, path_to_clone + relative_path));
disk->removeRecursive(fs::path(path_to_clone) / relative_path / "");
}
disk->createDirectories(path_to_clone);

View File

@ -28,9 +28,11 @@ std::pair<bool, ReplicatedMergeMutateTaskBase::PartLogWriter> MutateFromLogEntry
if (source_part->name != source_part_name)
{
LOG_WARNING(log, "Part " + source_part_name + " is covered by " + source_part->name
+ " but should be mutated to " + entry.new_part_name + ". "
+ "Possibly the mutation of this part is not needed and will be skipped. This shouldn't happen often.");
LOG_WARNING(log,
"Part {} is covered by {} but should be mutated to {}. "
"Possibly the mutation of this part is not needed and will be skipped. "
"This shouldn't happen often.",
source_part_name, source_part->name, entry.new_part_name);
return {false, {}};
}

View File

@ -2203,7 +2203,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr
if (replica.empty())
throw Exception(ErrorCodes::NO_REPLICA_HAS_PART, "Not found active replica on shard {} to clone part {}", entry.source_shard, entry.new_part_name);
LOG_INFO(log, "Will clone part from shard " + entry.source_shard + " and replica " + replica);
LOG_INFO(log, "Will clone part from shard {} and replica {}", entry.source_shard, replica);
MutableDataPartPtr part;
@ -6685,7 +6685,7 @@ void StorageReplicatedMergeTree::movePartitionToShard(
zkutil::KeeperMultiException::check(rc, ops, responses);
String task_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*responses.back()).path_created;
LOG_DEBUG(log, "Created task for part movement between shards at " + task_znode_path);
LOG_DEBUG(log, "Created task for part movement between shards at {}", task_znode_path);
/// TODO(nv): Nice to have support for `replication_alter_partitions_sync`.
/// For now use the system.part_moves_between_shards table for status.
@ -6824,7 +6824,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI
bool StorageReplicatedMergeTree::dropPartImpl(
zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop)
{
LOG_TRACE(log, "Will try to insert a log entry to DROP_RANGE for part: " + part_name);
LOG_TRACE(log, "Will try to insert a log entry to DROP_RANGE for part {}", part_name);
auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version);