mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
parent
f68d1ceb4f
commit
f69cbdcbfc
@ -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() {}
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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"))
|
||||
{
|
||||
|
@ -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.");
|
||||
|
@ -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());
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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))
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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));
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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()),
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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(
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user