find {base,src,programs} -name '*.h' -or -name '*.cpp' | xargs grep -l -P 'LOG_\w+\([^,]+, "[^"]+" \+ [^+]+ \+ "[^"]+"\);' | xargs sed -i -r -e 's/(LOG_\w+)\(([^,]+), "([^"]+)" \+ ([^+]+) \+ "([^"]+)"\);/\1_FORMATTED(\2, "\3{}\5", \4);/'

This commit is contained in:
Alexey Milovidov 2020-05-23 20:10:21 +03:00
parent f69cbdcbfc
commit 241f8c5431
10 changed files with 20 additions and 20 deletions

View File

@ -367,7 +367,7 @@ bool DiskAccessStorage::readLists()
auto file_path = getListFilePath(directory_path, type);
if (!std::filesystem::exists(file_path))
{
LOG_WARNING(getLogger(), "File " + file_path.string() + " doesn't exist");
LOG_WARNING_FORMATTED(getLogger(), "File {} doesn't exist", file_path.string());
ok = false;
break;
}

View File

@ -440,7 +440,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
zkutil::ZooKeeperNodeCache * zk_node_cache,
const zkutil::EventPtr & zk_changed_event)
{
LOG_DEBUG(log, "Processing configuration file '" + path + "'.");
LOG_DEBUG_FORMATTED(log, "Processing configuration file '{}'.", path);
XMLDocumentPtr config = dom_parser.parse(path);
@ -451,7 +451,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
{
try
{
LOG_DEBUG(log, "Merging configuration file '" + merge_file + "'.");
LOG_DEBUG_FORMATTED(log, "Merging configuration file '{}'.", merge_file);
XMLDocumentPtr with = dom_parser.parse(merge_file);
merge(config, with);
@ -488,7 +488,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
}
if (!include_from_path.empty())
{
LOG_DEBUG(log, "Including configuration file '" + include_from_path + "'.");
LOG_DEBUG_FORMATTED(log, "Including configuration file '{}'.", include_from_path);
contributing_files.push_back(include_from_path);
include_from = dom_parser.parse(include_from_path);

View File

@ -36,7 +36,7 @@ Exception::Exception(const std::string & msg, int code)
#ifndef NDEBUG
if (code == ErrorCodes::LOGICAL_ERROR)
{
LOG_ERROR(&Poco::Logger::root(), "Logical error: '" + msg + "'.");
LOG_ERROR_FORMATTED(&Poco::Logger::root(), "Logical error: '{}'.", msg);
assert(false);
}
#endif

View File

@ -312,7 +312,7 @@ DiskLocalReservation::~DiskLocalReservation()
if (disk->reserved_bytes < size)
{
disk->reserved_bytes = 0;
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '" + disk->getName() + "'.");
LOG_ERROR_FORMATTED(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
}
else
{
@ -320,7 +320,7 @@ DiskLocalReservation::~DiskLocalReservation()
}
if (disk->reservation_count == 0)
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '" + disk->getName() + "'.");
LOG_ERROR_FORMATTED(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
else
--disk->reservation_count;
}

View File

@ -677,7 +677,7 @@ DiskS3Reservation::~DiskS3Reservation()
if (disk->reserved_bytes < size)
{
disk->reserved_bytes = 0;
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '" + disk->getName() + "'.");
LOG_ERROR_FORMATTED(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
}
else
{
@ -685,7 +685,7 @@ DiskS3Reservation::~DiskS3Reservation()
}
if (disk->reservation_count == 0)
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '" + disk->getName() + "'.");
LOG_ERROR_FORMATTED(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
else
--disk->reservation_count;
}

View File

@ -701,7 +701,7 @@ void DDLWorker::checkShardConfig(const String & table, const DDLTask & task, Sto
if (dynamic_cast<const StorageDistributed *>(storage.get()))
{
LOG_TRACE(log, "Table " + backQuote(table) + " is distributed, skip checking config.");
LOG_TRACE_FORMATTED(log, "Table {} is distributed, skip checking config.", backQuote(table));
return;
}

View File

@ -250,7 +250,7 @@ private:
{
if (path.empty() || !repository.exists(path))
{
LOG_WARNING(log, "Config file '" + path + "' does not exist");
LOG_WARNING_FORMATTED(log, "Config file '{}' does not exist", path);
return false;
}

View File

@ -401,12 +401,12 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery &)
LOG_TRACE_FORMATTED(log, "Synchronizing entries in replica's queue with table's log and waiting for it to become empty");
if (!storage_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.totalMilliseconds()))
{
LOG_ERROR(log, "SYNC REPLICA " + table_id.getNameForLogs() + ": Timed out!");
LOG_ERROR_FORMATTED(log, "SYNC REPLICA {}: Timed out!", table_id.getNameForLogs());
throw Exception(
"SYNC REPLICA " + table_id.getNameForLogs() + ": command timed out! "
"See the 'receive_timeout' setting", ErrorCodes::TIMEOUT_EXCEEDED);
}
LOG_TRACE(log, "SYNC REPLICA " + table_id.getNameForLogs() + ": OK");
LOG_TRACE_FORMATTED(log, "SYNC REPLICA {}: OK", table_id.getNameForLogs());
}
else
throw Exception("Table " + table_id.getNameForLogs() + " is not replicated", ErrorCodes::BAD_ARGUMENTS);

View File

@ -246,7 +246,7 @@ void SystemLog<LogElement>::add(const LogElement & element)
requested_flush_before = queue_end;
flush_event.notify_all();
LOG_INFO(log, "Queue is half full for system log '" + demangle(typeid(*this).name()) + "'.");
LOG_INFO_FORMATTED(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name()));
}
if (queue.size() >= DBMS_SYSTEM_LOG_QUEUE_SIZE)

View File

@ -618,11 +618,11 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C
{
if (!it->second.is_done)
{
LOG_DEBUG(log, "Removing killed mutation " + entry.znode_name + " from local state.");
LOG_DEBUG_FORMATTED(log, "Removing killed mutation {} from local state.", entry.znode_name);
some_active_mutations_were_killed = true;
}
else
LOG_DEBUG(log, "Removing obsolete mutation " + entry.znode_name + " from local state.");
LOG_DEBUG_FORMATTED(log, "Removing obsolete mutation {} from local state.", entry.znode_name);
for (const auto & partition_and_block_num : entry.block_numbers)
{
@ -727,7 +727,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation(
auto rc = zookeeper->tryRemove(zookeeper_path + "/mutations/" + mutation_id);
if (rc == Coordination::ZOK)
LOG_DEBUG(log, "Removed mutation " + mutation_id + " from ZooKeeper.");
LOG_DEBUG_FORMATTED(log, "Removed mutation {} from ZooKeeper.", mutation_id);
ReplicatedMergeTreeMutationEntryPtr entry;
bool mutation_was_active = false;
@ -756,7 +756,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation(
}
mutations_by_znode.erase(it);
LOG_DEBUG(log, "Removed mutation " + entry->znode_name + " from local state.");
LOG_DEBUG_FORMATTED(log, "Removed mutation {} from local state.", entry->znode_name);
}
if (mutation_was_active)
@ -1161,13 +1161,13 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting()
for (const String & new_part_name : entry->getBlockingPartNames())
{
if (!queue.future_parts.erase(new_part_name))
LOG_ERROR(queue.log, "Untagging already untagged future part " + new_part_name + ". This is a bug.");
LOG_ERROR_FORMATTED(queue.log, "Untagging already untagged future part {}. This is a bug.", new_part_name);
}
if (!entry->actual_new_part_name.empty())
{
if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name))
LOG_ERROR(queue.log, "Untagging already untagged future part " + entry->actual_new_part_name + ". This is a bug.");
LOG_ERROR_FORMATTED(queue.log, "Untagging already untagged future part {}. This is a bug.", entry->actual_new_part_name);
entry->actual_new_part_name.clear();
}