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{}", \4);/'

This commit is contained in:
Alexey Milovidov 2020-05-23 20:09:37 +03:00
parent f68d1ceb4f
commit f69cbdcbfc
19 changed files with 34 additions and 34 deletions

View File

@ -502,7 +502,7 @@ void debugIncreaseOOMScore()
e.displayText() + "'.");
return;
}
LOG_INFO(&Logger::root(), "Set OOM score adjustment to " + new_score);
LOG_INFO_FORMATTED(&Logger::root(), "Set OOM score adjustment to {}", new_score);
}
#else
void debugIncreaseOOMScore() {}

View File

@ -62,7 +62,7 @@ namespace
void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
{
Poco::Net::HTMLForm params(request, request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
LOG_TRACE_FORMATTED(log, "Request URI: {}", request.getURI());
auto process_error = [&response, this](const std::string & message)
{

View File

@ -10,7 +10,7 @@ namespace DB
Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request)
{
Poco::URI uri{request.getURI()};
LOG_TRACE(log, "Request URI: " + uri.toString());
LOG_TRACE_FORMATTED(log, "Request URI: {}", uri.toString());
if (uri.getPath() == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
return new PingHandler(keep_alive_timeout);

View File

@ -25,7 +25,7 @@ namespace DB
void IdentifierQuoteHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
{
Poco::Net::HTMLForm params(request, request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
LOG_TRACE_FORMATTED(log, "Request URI: {}", request.getURI());
auto process_error = [&response, this](const std::string & message)
{

View File

@ -84,7 +84,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
Poco::Net::HTMLForm params(request);
if (mode == "read")
params.read(request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
LOG_TRACE_FORMATTED(log, "Request URI: {}", request.getURI());
if (mode == "read" && !params.has("query"))
{

View File

@ -188,7 +188,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
new HandlerFactory("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context), server_pool, socket, http_params);
server.start();
LOG_INFO(log, "Listening http://" + address.toString());
LOG_INFO_FORMATTED(log, "Listening http://{}", address.toString());
SCOPE_EXIT({
LOG_DEBUG_FORMATTED(log, "Received termination signal.");

View File

@ -276,7 +276,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
LOG_TRACE_FORMATTED(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
if (0 != mlockall(MCL_CURRENT))
LOG_WARNING(log, "Failed mlockall: " + errnoToString(ErrorCodes::SYSTEM_ERROR));
LOG_WARNING_FORMATTED(log, "Failed mlockall: {}", errnoToString(ErrorCodes::SYSTEM_ERROR));
else
LOG_TRACE_FORMATTED(log, "The memory map of clickhouse executable has been mlock'ed");
}
@ -592,7 +592,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
LOG_INFO(log, "Loading metadata from " + path);
LOG_INFO_FORMATTED(log, "Loading metadata from {}", path);
try
{
@ -826,7 +826,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for http://" + address.toString());
LOG_INFO_FORMATTED(log, "Listening for http://{}", address.toString());
});
/// HTTPS
@ -840,7 +840,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for https://" + address.toString());
LOG_INFO_FORMATTED(log, "Listening for https://{}", address.toString());
#else
UNUSED(port);
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
@ -861,7 +861,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections with native protocol (tcp): " + address.toString());
LOG_INFO_FORMATTED(log, "Listening for connections with native protocol (tcp): {}", address.toString());
});
/// TCP with SSL
@ -877,7 +877,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
server_pool,
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): " + address.toString());
LOG_INFO_FORMATTED(log, "Listening for connections with secure native protocol (tcp_secure): {}", address.toString());
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
@ -895,7 +895,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for replica communication (interserver): http://" + address.toString());
LOG_INFO_FORMATTED(log, "Listening for replica communication (interserver): http://{}", address.toString());
});
create_server("interserver_https_port", [&](UInt16 port)
@ -908,7 +908,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for secure replica communication (interserver): https://" + address.toString());
LOG_INFO_FORMATTED(log, "Listening for secure replica communication (interserver): https://{}", address.toString());
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
@ -928,7 +928,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for MySQL compatibility protocol: " + address.toString());
LOG_INFO_FORMATTED(log, "Listening for MySQL compatibility protocol: {}", address.toString());
});
/// Prometheus (if defined and not setup yet with http_port)
@ -941,7 +941,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
LOG_INFO(log, "Listening for Prometheus: http://" + address.toString());
LOG_INFO_FORMATTED(log, "Listening for Prometheus: http://{}", address.toString());
});
}

View File

@ -496,7 +496,7 @@ void DiskAccessStorage::listsWritingThreadFunc()
/// and then saves the files "users.list", "roles.list", etc. to the same directory.
bool DiskAccessStorage::rebuildLists()
{
LOG_WARNING(getLogger(), "Recovering lists in directory " + directory_path);
LOG_WARNING_FORMATTED(getLogger(), "Recovering lists in directory {}", directory_path);
clear();
for (const auto & directory_entry : std::filesystem::directory_iterator(directory_path))

View File

@ -165,10 +165,10 @@ void QueryProfilerBase<ProfilerImpl>::tryCleanup()
{
#if USE_UNWIND
if (timer_id != nullptr && timer_delete(timer_id))
LOG_ERROR(log, "Failed to delete query profiler timer " + errnoToString(ErrorCodes::CANNOT_DELETE_TIMER));
LOG_ERROR_FORMATTED(log, "Failed to delete query profiler timer {}", errnoToString(ErrorCodes::CANNOT_DELETE_TIMER));
if (previous_handler != nullptr && sigaction(pause_signal, previous_handler, nullptr))
LOG_ERROR(log, "Failed to restore signal handler after query profiler " + errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER));
LOG_ERROR_FORMATTED(log, "Failed to restore signal handler after query profiler {}", errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER));
#endif
}

View File

@ -309,7 +309,7 @@ struct ODBCBridgeMixin
cmd_args.push_back(config.getString("logger." + configPrefix() + "_level"));
}
LOG_TRACE(log, "Starting " + serviceAlias());
LOG_TRACE_FORMATTED(log, "Starting {}", serviceAlias());
return ShellCommand::executeDirect(path.toString(), cmd_args, true);
}

View File

@ -95,10 +95,10 @@ Block MergeSortingBlockInputStream::readImpl()
const std::string & path = temporary_files.back()->path();
MergeSortingBlocksBlockInputStream block_in(blocks, description, max_merged_block_size, limit);
LOG_INFO(log, "Sorting and writing part of data into temporary file " + path);
LOG_INFO_FORMATTED(log, "Sorting and writing part of data into temporary file {}", path);
ProfileEvents::increment(ProfileEvents::ExternalSortWritePart);
TemporaryFileStream::write(path, header_without_constants, block_in, &is_cancelled, codec); /// NOTE. Possibly limit disk usage.
LOG_INFO(log, "Done writing part of data into temporary file " + path);
LOG_INFO_FORMATTED(log, "Done writing part of data into temporary file {}", path);
blocks.clear();
sum_bytes_in_blocks = 0;

View File

@ -77,7 +77,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar
BlockInputStreamPtr ExecutableDictionarySource::loadAll()
{
LOG_TRACE(log, "loadAll " + toString());
LOG_TRACE_FORMATTED(log, "loadAll {}", toString());
auto process = ShellCommand::execute(command);
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<ShellCommandOwningBlockInputStream>(input_stream, std::move(process));
@ -92,7 +92,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll()
if (update_time)
command_with_update_field += " " + update_field + " " + DB::toString(LocalDateTime(update_time - 1));
LOG_TRACE(log, "loadUpdatedAll " + command_with_update_field);
LOG_TRACE_FORMATTED(log, "loadUpdatedAll {}", command_with_update_field);
auto process = ShellCommand::execute(command_with_update_field);
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<ShellCommandOwningBlockInputStream>(input_stream, std::move(process));

View File

@ -49,7 +49,7 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other)
BlockInputStreamPtr FileDictionarySource::loadAll()
{
LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll " + toString());
LOG_TRACE_FORMATTED(&Poco::Logger::get("FileDictionary"), "loadAll {}", toString());
auto in_ptr = std::make_unique<ReadBufferFromFile>(filepath);
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
last_modification = getLastModification();

View File

@ -105,7 +105,7 @@ void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri)
BlockInputStreamPtr HTTPDictionarySource::loadAll()
{
LOG_TRACE(log, "loadAll " + toString());
LOG_TRACE_FORMATTED(log, "loadAll {}", toString());
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts,
@ -118,7 +118,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll()
{
Poco::URI uri(url);
getUpdateFieldAndDate(uri);
LOG_TRACE(log, "loadUpdatedAll " + uri.toString());
LOG_TRACE_FORMATTED(log, "loadUpdatedAll {}", uri.toString());
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts,
0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries);

View File

@ -182,7 +182,7 @@ LibraryDictionarySource::~LibraryDictionarySource()
BlockInputStreamPtr LibraryDictionarySource::loadAll()
{
LOG_TRACE(log, "loadAll " + toString());
LOG_TRACE_FORMATTED(log, "loadAll {}", toString());
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(dict_struct.attributes.size());
ClickHouseLibrary::CStrings columns{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()),

View File

@ -635,7 +635,7 @@ void DatabaseCatalog::dropTableDataTask()
if (it != tables_marked_dropped.end())
{
table = std::move(*it);
LOG_INFO(log, "Will try drop " + table.table_id.getNameForLogs());
LOG_INFO_FORMATTED(log, "Will try drop {}", table.table_id.getNameForLogs());
tables_marked_dropped.erase(it);
}
need_reschedule = !tables_marked_dropped.empty();

View File

@ -36,7 +36,7 @@ public:
, path(std::move(path_)), file_buf_out(path), compressed_buf_out(file_buf_out)
, out_stream(std::make_shared<NativeBlockOutputStream>(compressed_buf_out, 0, header))
{
LOG_INFO(log, "Sorting and writing part of data into temporary file " + path);
LOG_INFO_FORMATTED(log, "Sorting and writing part of data into temporary file {}", path);
ProfileEvents::increment(ProfileEvents::ExternalSortWritePart);
out_stream->writePrefix();
}
@ -55,7 +55,7 @@ public:
out_stream->writeSuffix();
compressed_buf_out.next();
file_buf_out.next();
LOG_INFO(log, "Done writing part of data into temporary file " + path);
LOG_INFO_FORMATTED(log, "Done writing part of data into temporary file {}", path);
out_stream.reset();

View File

@ -2477,9 +2477,9 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String &
partition_id = getPartitionIDFromQuery(partition_ast, context);
if (prefix)
LOG_DEBUG(log, "Freezing parts with prefix " + *prefix);
LOG_DEBUG_FORMATTED(log, "Freezing parts with prefix {}", *prefix);
else
LOG_DEBUG(log, "Freezing parts with partition ID " + partition_id);
LOG_DEBUG_FORMATTED(log, "Freezing parts with partition ID {}", partition_id);
freezePartitionsByMatcher(

View File

@ -252,7 +252,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
if (new_data_part->volume->getDisk()->exists(full_path))
{
LOG_WARNING(log, "Removing old temporary directory " + fullPath(new_data_part->volume->getDisk(), full_path));
LOG_WARNING_FORMATTED(log, "Removing old temporary directory {}", fullPath(new_data_part->volume->getDisk(), full_path));
new_data_part->volume->getDisk()->removeRecursive(full_path);
}