From bedf208cbd3243982833eb5468827a84d579ec95 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Feb 2022 12:10:27 +0300 Subject: [PATCH] Use fmt::runtime() for LOG_* for non constexpr Here is oneliner: $ gg 'LOG_\(DEBUG\|TRACE\|INFO\|TEST\|WARNING\|ERROR\|FATAL\)([^,]*, [a-zA-Z]' -- :*.cpp :*.h | cut -d: -f1 | sort -u | xargs -r sed -E -i 's#(LOG_[A-Z]*)\(([^,]*), ([A-Za-z][^,)]*)#\1(\2, fmt::runtime(\3)#' Note, that I tried to do this with coccinelle (tool for semantic patchin), but it cannot parse C++: $ cat fmt.cocci @@ expression log; expression var; @@ -LOG_DEBUG(log, var) +LOG_DEBUG(log, fmt::runtime(var)) I've also tried to use some macros/templates magic to do this implicitly in logger_useful.h, but I failed to do so, and apparently it is not possible for now. Signed-off-by: Azat Khuzhin v2: manual fixes Signed-off-by: Azat Khuzhin --- base/daemon/BaseDaemon.cpp | 6 ++--- programs/keeper/Keeper.cpp | 2 +- programs/library-bridge/Handlers.cpp | 6 ++--- programs/odbc-bridge/ColumnInfoHandler.cpp | 2 +- .../odbc-bridge/IdentifierQuoteHandler.cpp | 2 +- programs/odbc-bridge/MainHandler.cpp | 4 ++-- programs/odbc-bridge/SchemaAllowedHandler.cpp | 2 +- src/Client/ClientBase.cpp | 2 +- src/Client/Connection.cpp | 2 +- src/Client/ConnectionEstablisher.cpp | 6 ++--- src/Common/DNSResolver.cpp | 2 +- src/Coordination/LoggerWrapper.h | 2 +- src/Databases/DatabaseAtomic.cpp | 8 +++---- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/SQLite/SQLiteUtils.cpp | 2 +- src/Dictionaries/CassandraHelpers.cpp | 10 ++++---- src/Dictionaries/MySQLDictionarySource.cpp | 6 ++--- .../PostgreSQLDictionarySource.cpp | 4 ++-- src/Dictionaries/XDBCDictionarySource.cpp | 4 ++-- src/Disks/DiskSelector.cpp | 2 +- src/Functions/logTrace.cpp | 2 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Parsers/DumpASTNode.h | 2 +- src/Server/GRPCServer.cpp | 2 +- src/Server/HTTPHandler.cpp | 5 +++- src/Server/InterserverIOHTTPHandler.cpp | 6 ++--- src/Server/PostgreSQLHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- .../MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../ReplicatedMergeMutateTaskBase.cpp | 6 ++--- .../ReplicatedMergeTreePartCheckThread.cpp | 4 ++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 24 +++++++++---------- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 18 +++++++------- 36 files changed, 82 insertions(+), 79 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index f3026d7c87a..0c3acc4a3df 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -317,7 +317,7 @@ private: else error_message = "Sanitizer trap."; - LOG_FATAL(log, error_message); + LOG_FATAL(log, fmt::runtime(error_message)); if (stack_trace.getSize()) { @@ -330,11 +330,11 @@ private: for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i) bare_stacktrace << ' ' << stack_trace.getFramePointers()[i]; - LOG_FATAL(log, bare_stacktrace.str()); + LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); } /// Write symbolized stack trace line by line for better grep-ability. - stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, s); }); + stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, fmt::runtime(s)); }); #if defined(OS_LINUX) /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 636ce129d63..88df4d5b3e7 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -324,7 +324,7 @@ int Keeper::main(const std::vector & /*args*/) } else { - LOG_WARNING(log, message); + LOG_WARNING(log, fmt::runtime(message)); } } diff --git a/programs/library-bridge/Handlers.cpp b/programs/library-bridge/Handlers.cpp index bf9ace679ba..d16ac18ab56 100644 --- a/programs/library-bridge/Handlers.cpp +++ b/programs/library-bridge/Handlers.cpp @@ -37,7 +37,7 @@ namespace if (!response.sent()) *response.send() << message << std::endl; - LOG_WARNING(&Poco::Logger::get("LibraryBridge"), message); + LOG_WARNING(&Poco::Logger::get("LibraryBridge"), fmt::runtime(message)); } std::shared_ptr parseColumns(std::string && column_string) @@ -178,7 +178,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe catch (const Exception & ex) { processError(response, "Invalid 'sample_block' parameter in request body '" + ex.message() + "'"); - LOG_WARNING(log, ex.getStackTraceString()); + LOG_WARNING(log, fmt::runtime(ex.getStackTraceString())); return; } @@ -278,7 +278,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe catch (const Exception & ex) { processError(response, "Invalid 'requested_block' parameter in request body '" + ex.message() + "'"); - LOG_WARNING(log, ex.getStackTraceString()); + LOG_WARNING(log, fmt::runtime(ex.getStackTraceString())); return; } diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 8ceeddcd7ab..4d9a6b7a692 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -77,7 +77,7 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); if (!response.sent()) *response.send() << message << std::endl; - LOG_WARNING(log, message); + LOG_WARNING(log, fmt::runtime(message)); }; if (!params.has("table")) diff --git a/programs/odbc-bridge/IdentifierQuoteHandler.cpp b/programs/odbc-bridge/IdentifierQuoteHandler.cpp index c7cad68f19e..7f809da4b10 100644 --- a/programs/odbc-bridge/IdentifierQuoteHandler.cpp +++ b/programs/odbc-bridge/IdentifierQuoteHandler.cpp @@ -29,7 +29,7 @@ void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServ response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); if (!response.sent()) *response.send() << message << std::endl; - LOG_WARNING(log, message); + LOG_WARNING(log, fmt::runtime(message)); }; if (!params.has("connection_string")) diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 1252d1ae70a..02bdabe8ffa 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -46,7 +46,7 @@ void ODBCHandler::processError(HTTPServerResponse & response, const std::string response.setStatusAndReason(HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); if (!response.sent()) *response.send() << message << std::endl; - LOG_WARNING(log, message); + LOG_WARNING(log, fmt::runtime(message)); } @@ -102,7 +102,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse catch (const Exception & ex) { processError(response, "Invalid 'sample_block' parameter in request body '" + ex.message() + "'"); - LOG_ERROR(log, ex.getStackTraceString()); + LOG_ERROR(log, fmt::runtime(ex.getStackTraceString())); return; } diff --git a/programs/odbc-bridge/SchemaAllowedHandler.cpp b/programs/odbc-bridge/SchemaAllowedHandler.cpp index 7b526bd8041..0c58af2f7c1 100644 --- a/programs/odbc-bridge/SchemaAllowedHandler.cpp +++ b/programs/odbc-bridge/SchemaAllowedHandler.cpp @@ -37,7 +37,7 @@ void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServer response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); if (!response.sent()) *response.send() << message << std::endl; - LOG_WARNING(log, message); + LOG_WARNING(log, fmt::runtime(message)); }; if (!params.has("connection_string")) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 006307cb433..27deace416d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1261,7 +1261,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin for (const auto & query_id_format : query_id_formats) { writeString(query_id_format.first, std_out); - writeString(fmt::format(query_id_format.second, fmt::arg("query_id", global_context->getCurrentQueryId())), std_out); + writeString(fmt::format(fmt::runtime(query_id_format.second), fmt::arg("query_id", global_context->getCurrentQueryId())), std_out); writeChar('\n', std_out); std_out.next(); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 505a6514812..ad2fc76f090 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -405,7 +405,7 @@ bool Connection::ping() } catch (const Poco::Exception & e) { - LOG_TRACE(log_wrapper.get(), e.displayText()); + LOG_TRACE(log_wrapper.get(), fmt::runtime(e.displayText())); return false; } diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 4d27c9efc69..3385834e386 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -58,9 +58,9 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: auto table_status_it = status_response.table_states_by_id.find(*table_to_check); if (table_status_it == status_response.table_states_by_id.end()) { - const char * message_pattern = "There is no table {}.{} on server: {}"; - fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); - LOG_WARNING(log, fail_message); + fail_message = fmt::format("There is no table {}.{} on server: {}", + backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); + LOG_WARNING(log, fmt::runtime(fail_message)); ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); return; } diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 36d0c13b153..13da3efd57a 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -272,7 +272,7 @@ bool DNSResolver::updateCacheImpl(UpdateF && update_func, ElemsT && elems, const } if (!lost_elems.empty()) - LOG_INFO(log, log_msg, lost_elems); + LOG_INFO(log, fmt::runtime(log_msg), lost_elems); return updated; } diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index 002fa870241..a2493763633 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -39,7 +39,7 @@ public: const std::string & msg) override { LogsLevel db_level = static_cast(level_); - LOG_IMPL(log, db_level, LEVELS.at(db_level), msg); + LOG_IMPL(log, db_level, LEVELS.at(db_level), fmt::runtime(msg)); } void set_level(int level_) override diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index cb0c1cdae95..721bf79199b 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -80,7 +80,7 @@ void DatabaseAtomic::drop(ContextPtr) } catch (...) { - LOG_WARNING(log, getCurrentExceptionMessage(true)); + LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); } fs::remove_all(getMetadataPath()); } @@ -469,7 +469,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & } catch (...) { - LOG_WARNING(log, getCurrentExceptionMessage(true)); + LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); } } @@ -482,7 +482,7 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name) } catch (...) { - LOG_WARNING(log, getCurrentExceptionMessage(true)); + LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); } } @@ -527,7 +527,7 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new } catch (...) { - LOG_WARNING(log, getCurrentExceptionMessage(true)); + LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); } auto new_name_escaped = escapeForFileName(new_name); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 165bad950f5..29591a5f88f 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -316,7 +316,7 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na } catch (...) { - LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); + LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(__PRETTY_FUNCTION__))); attachTable(local_context, table_name, table, table_data_path_relative); if (renamed) fs::rename(table_metadata_path_drop, table_metadata_path); diff --git a/src/Databases/SQLite/SQLiteUtils.cpp b/src/Databases/SQLite/SQLiteUtils.cpp index 954576d9c05..5b38caeabee 100644 --- a/src/Databases/SQLite/SQLiteUtils.cpp +++ b/src/Databases/SQLite/SQLiteUtils.cpp @@ -20,7 +20,7 @@ void processSQLiteError(const String & message, bool throw_on_error) if (throw_on_error) throw Exception(ErrorCodes::PATH_ACCESS_DENIED, message); else - LOG_ERROR(&Poco::Logger::get("SQLiteEngine"), message); + LOG_ERROR(&Poco::Logger::get("SQLiteEngine"), fmt::runtime(message)); } diff --git a/src/Dictionaries/CassandraHelpers.cpp b/src/Dictionaries/CassandraHelpers.cpp index a33ab288a34..235e29b5bd8 100644 --- a/src/Dictionaries/CassandraHelpers.cpp +++ b/src/Dictionaries/CassandraHelpers.cpp @@ -58,15 +58,15 @@ void cassandraLogCallback(const CassLogMessage * message, void * data) { Poco::Logger * logger = static_cast(data); if (message->severity == CASS_LOG_CRITICAL || message->severity == CASS_LOG_ERROR) - LOG_ERROR(logger, message->message); + LOG_ERROR(logger, fmt::runtime(message->message)); else if (message->severity == CASS_LOG_WARN) - LOG_WARNING(logger, message->message); + LOG_WARNING(logger, fmt::runtime(message->message)); else if (message->severity == CASS_LOG_INFO) - LOG_INFO(logger, message->message); + LOG_INFO(logger, fmt::runtime(message->message)); else if (message->severity == CASS_LOG_DEBUG) - LOG_DEBUG(logger, message->message); + LOG_DEBUG(logger, fmt::runtime(message->message)); else if (message->severity == CASS_LOG_TRACE) - LOG_TRACE(logger, message->message); + LOG_TRACE(logger, fmt::runtime(message->message)); } } diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index a291fcea47f..29d70f3a7c4 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -193,7 +193,7 @@ Pipe MySQLDictionarySource::loadAll() auto connection = pool->get(); last_modification = getLastModification(connection, false); - LOG_TRACE(log, load_all_query); + LOG_TRACE(log, fmt::runtime(load_all_query)); return loadFromQuery(load_all_query); } @@ -203,7 +203,7 @@ Pipe MySQLDictionarySource::loadUpdatedAll() last_modification = getLastModification(connection, false); std::string load_update_query = getUpdateFieldAndDate(); - LOG_TRACE(log, load_update_query); + LOG_TRACE(log, fmt::runtime(load_update_query)); return loadFromQuery(load_update_query); } @@ -289,7 +289,7 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry & { auto query = connection->query("SHOW TABLE STATUS LIKE " + quoteForLike(configuration.table)); - LOG_TRACE(log, query.str()); + LOG_TRACE(log, fmt::runtime(query.str())); auto result = query.use(); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 9af3ea06838..6fdf486fdbf 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -80,7 +80,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar Pipe PostgreSQLDictionarySource::loadAll() { - LOG_TRACE(log, load_all_query); + LOG_TRACE(log, fmt::runtime(load_all_query)); return loadBase(load_all_query); } @@ -88,7 +88,7 @@ Pipe PostgreSQLDictionarySource::loadAll() Pipe PostgreSQLDictionarySource::loadUpdatedAll() { auto load_update_query = getUpdateFieldAndDate(); - LOG_TRACE(log, load_update_query); + LOG_TRACE(log, fmt::runtime(load_update_query)); return loadBase(load_update_query); } diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index e95094cac47..f08abcdc516 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -121,7 +121,7 @@ std::string XDBCDictionarySource::getUpdateFieldAndDate() Pipe XDBCDictionarySource::loadAll() { - LOG_TRACE(log, load_all_query); + LOG_TRACE(log, fmt::runtime(load_all_query)); return loadFromQuery(bridge_url, sample_block, load_all_query); } @@ -130,7 +130,7 @@ Pipe XDBCDictionarySource::loadUpdatedAll() { std::string load_query_update = getUpdateFieldAndDate(); - LOG_TRACE(log, load_query_update); + LOG_TRACE(log, fmt::runtime(load_query_update)); return loadFromQuery(bridge_url, sample_block, load_query_update); } diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index df77006addc..4c80b128b4b 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -101,7 +101,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig( } writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); - LOG_WARNING(&Poco::Logger::get("DiskSelector"), warning.str()); + LOG_WARNING(&Poco::Logger::get("DiskSelector"), fmt::runtime(warning.str())); } return result; diff --git a/src/Functions/logTrace.cpp b/src/Functions/logTrace.cpp index acf2a2041ec..05315f4dff6 100644 --- a/src/Functions/logTrace.cpp +++ b/src/Functions/logTrace.cpp @@ -48,7 +48,7 @@ namespace "First argument for function " + getName() + " must be Constant string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); static auto * log = &Poco::Logger::get("FunctionLogTrace"); - LOG_TRACE(log, message); + LOG_TRACE(log, fmt::runtime(message)); return DataTypeUInt8().createColumnConst(input_rows_count, 0); } diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 629c15cdae0..ce00676b2ed 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -325,7 +325,7 @@ Chunk DDLQueryStatusSource::generate() return {}; } - LOG_INFO(log, msg_format, node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts); + LOG_INFO(log, fmt::runtime(msg_format), node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts); NameSet unfinished_hosts = waiting_hosts; for (const auto & host_id : finished_hosts) diff --git a/src/Parsers/DumpASTNode.h b/src/Parsers/DumpASTNode.h index e1071f02dd3..e8efeb4b59c 100644 --- a/src/Parsers/DumpASTNode.h +++ b/src/Parsers/DumpASTNode.h @@ -102,7 +102,7 @@ public: ~DebugASTLog() { if constexpr (_enable) - LOG_DEBUG(log, buf.str()); + LOG_DEBUG(log, fmt::runtime(buf.str())); } WriteBuffer * stream() { return (_enable ? &buf : nullptr); } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 85b1d345c6e..0faafb92ad5 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1262,7 +1262,7 @@ namespace { io.onException(); - LOG_ERROR(log, getExceptionMessage(exception, true)); + LOG_ERROR(log, fmt::runtime(getExceptionMessage(exception, true))); if (responder && !responder_finished) { diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 5253e66be92..a42df54aed7 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -915,7 +915,10 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse } processQuery(request, params, response, used_output, query_scope); - LOG_DEBUG(log, (request_credentials ? "Authentication in progress..." : "Done processing query")); + if (request_credentials) + LOG_DEBUG(log, "Authentication in progress..."); + else + LOG_DEBUG(log, "Done processing query"); } catch (...) { diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 082f7cc2e33..9506c5c133f 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -138,9 +138,9 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe write_response(message); if (is_real_error) - LOG_ERROR(log, message); + LOG_ERROR(log, fmt::runtime(message)); else - LOG_INFO(log, message); + LOG_INFO(log, fmt::runtime(message)); } catch (...) { @@ -148,7 +148,7 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe std::string message = getCurrentExceptionMessage(false); write_response(message); - LOG_ERROR(log, message); + LOG_ERROR(log, fmt::runtime(message)); } } diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 9808b538280..d6b834fceb7 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -105,7 +105,7 @@ void PostgreSQLHandler::run() "0A000", "Command is not supported"), true); - LOG_ERROR(log, Poco::format("Command is not supported. Command code %d", static_cast(message_type))); + LOG_ERROR(log, "Command is not supported. Command code {:d}", static_cast(message_type)); message_transport->dropMessage(); } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 6fa2b25d181..49595a9c658 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -466,7 +466,7 @@ void TCPHandler::runImpl() } const auto & e = *exception; - LOG_ERROR(log, getExceptionMessage(e, true)); + LOG_ERROR(log, fmt::runtime(getExceptionMessage(e, true))); sendException(*exception, send_exception_with_stack_trace); } } diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 37167038f63..11a2e8b0b94 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -525,7 +525,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( { if (e.code() != ErrorCodes::S3_ERROR && e.code() != ErrorCodes::ZERO_COPY_REPLICATION_ERROR) throw; - LOG_WARNING(log, e.message() + " Will retry fetching part without zero-copy."); + LOG_WARNING(log, fmt::runtime(e.message() + " Will retry fetching part without zero-copy.")); /// Try again but without zero-copy return fetchPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts, user, password, interserver_scheme, throttler, to_detached, tmp_prefix_, nullptr, false, disk); diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index d92eaf85f3d..b0b8aad2841 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -84,7 +84,7 @@ std::pair MergeFromLogEntryT /// 3. We have two intersecting parts, both cover source_part_name. It's logical error. /// TODO Why 1 and 2 can happen? Do we need more assertions here or somewhere else? constexpr const char * message = "Part {} is covered by {} but should be merged into {}. This shouldn't happen often."; - LOG_WARNING(log, message, source_part_name, source_part_or_covering->name, entry.new_part_name); + LOG_WARNING(log, fmt::runtime(message), source_part_name, source_part_or_covering->name, entry.new_part_name); if (!source_part_or_covering->info.contains(MergeTreePartInfo::fromPartName(entry.new_part_name, storage.format_version))) throw Exception(ErrorCodes::LOGICAL_ERROR, message, source_part_name, source_part_or_covering->name, entry.new_part_name); return {false, {}}; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6db50724787..4a41358b1d1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5918,7 +5918,7 @@ ReservationPtr MergeTreeData::balancedReservation( writeCString("\nbalancer: \n", log_str); for (const auto & [disk_name, per_disk_parts] : disk_parts_for_logging) writeString(fmt::format(" {}: [{}]\n", disk_name, fmt::join(per_disk_parts, ", ")), log_str); - LOG_DEBUG(log, log_str.str()); + LOG_DEBUG(log, fmt::runtime(log_str.str())); if (ttl_infos) reserved_space = tryReserveSpacePreferringTTLRules( diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index db5ca15ce8a..880e729e534 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -45,17 +45,17 @@ bool ReplicatedMergeMutateTaskBase::executeStep() if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART) { /// If no one has the right part, probably not all replicas work; We will not write to log with Error level. - LOG_INFO(log, e.displayText()); + LOG_INFO(log, fmt::runtime(e.displayText())); } else if (e.code() == ErrorCodes::ABORTED) { /// Interrupted merge or downloading a part is not an error. - LOG_INFO(log, e.message()); + LOG_INFO(log, fmt::runtime(e.message())); } else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED) { /// Part cannot be added temporarily - LOG_INFO(log, e.displayText()); + LOG_INFO(log, fmt::runtime(e.displayText())); storage.cleanup_thread.wakeup(); } else diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index bc3dd093f36..af877bdbdf0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -359,7 +359,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na tryLogCurrentException(log, __PRETTY_FUNCTION__); String message = "Part " + part_name + " looks broken. Removing it and will try to fetch."; - LOG_ERROR(log, message); + LOG_ERROR(log, fmt::runtime(message)); /// Delete part locally. storage.forgetPartAndMoveToDetached(part, "broken"); @@ -378,7 +378,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed); String message = "Unexpected part " + part_name + " in filesystem. Removing."; - LOG_ERROR(log, message); + LOG_ERROR(log, fmt::runtime(message)); storage.forgetPartAndMoveToDetached(part, "unexpected"); return {part_name, false, message}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 2a8c4d32578..05b0f8821a9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1062,7 +1062,7 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const LogEntry & en const LogEntry & another_entry = *entry_for_same_part_it->second; const char * format_str = "Not executing log entry {} of type {} for part {} " "because another log entry {} of type {} for the same part ({}) is being processed. This shouldn't happen often."; - LOG_INFO(log, format_str, entry.znode_name, entry.type, entry.new_part_name, + LOG_INFO(log, fmt::runtime(format_str), entry.znode_name, entry.type, entry.new_part_name, another_entry.znode_name, another_entry.type, another_entry.new_part_name); out_reason = fmt::format(format_str, entry.znode_name, entry.type, entry.new_part_name, another_entry.znode_name, another_entry.type, another_entry.new_part_name); @@ -1088,7 +1088,7 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const LogEntry & en { const char * format_str = "Not executing log entry {} for part {} " "because it is covered by part {} that is currently executing."; - LOG_TRACE(log, format_str, entry.znode_name, new_part_name, future_part_elem.first); + LOG_TRACE(log, fmt::runtime(format_str), entry.znode_name, new_part_name, future_part_elem.first); out_reason = fmt::format(format_str, entry.znode_name, new_part_name, future_part_elem.first); return false; } @@ -1173,7 +1173,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( { const char * format_str = "Not executing log entry {} of type {} for part {} " "because part {} is not ready yet (log entry for that part is being processed)."; - LOG_TRACE(log, format_str, entry.znode_name, entry.typeToString(), entry.new_part_name, name); + LOG_TRACE(log, fmt::runtime(format_str), entry.znode_name, entry.typeToString(), entry.new_part_name, name); /// Copy-paste of above because we need structured logging (instead of already formatted message). out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.typeToString(), entry.new_part_name, name); return false; @@ -1192,7 +1192,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (merger_mutator.merges_blocker.isCancelled()) { const char * format_str = "Not executing log entry {} of type {} for part {} because merges and mutations are cancelled now."; - LOG_DEBUG(log, format_str, entry.znode_name, entry.typeToString(), entry.new_part_name); + LOG_DEBUG(log, fmt::runtime(format_str), entry.znode_name, entry.typeToString(), entry.new_part_name); out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.typeToString(), entry.new_part_name); return false; } @@ -1229,7 +1229,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (merger_mutator.ttl_merges_blocker.isCancelled()) { const char * format_str = "Not executing log entry {} for part {} because merges with TTL are cancelled now."; - LOG_DEBUG(log, format_str, + LOG_DEBUG(log, fmt::runtime(format_str), entry.znode_name, entry.new_part_name); out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.new_part_name); return false; @@ -1239,7 +1239,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( { const char * format_str = "Not executing log entry {} for part {}" " because {} merges with TTL already executing, maximum {}."; - LOG_DEBUG(log, format_str, + LOG_DEBUG(log, fmt::runtime(format_str), entry.znode_name, entry.new_part_name, total_merges_with_ttl, data_settings->max_number_of_merges_with_ttl_in_pool); @@ -1258,7 +1258,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( entry.znode_name, entry.typeToString(), entry.new_part_name, ReadableSize(sum_parts_size_in_bytes), ReadableSize(max_source_parts_size)); - LOG_DEBUG(log, out_postpone_reason); + LOG_DEBUG(log, fmt::runtime(out_postpone_reason)); return false; } @@ -1272,7 +1272,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( { int head_alter = alter_sequence.getHeadAlterVersion(state_lock); const char * format_str = "Cannot execute alter metadata {} with version {} because another alter {} must be executed before"; - LOG_TRACE(log, format_str, entry.znode_name, entry.alter_version, head_alter); + LOG_TRACE(log, fmt::runtime(format_str), entry.znode_name, entry.alter_version, head_alter); out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.alter_version, head_alter); return false; } @@ -1287,13 +1287,13 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (head_alter == entry.alter_version) { const char * format_str = "Cannot execute alter data {} with version {} because metadata still not altered"; - LOG_TRACE(log, format_str, entry.znode_name, entry.alter_version); + LOG_TRACE(log, fmt::runtime(format_str), entry.znode_name, entry.alter_version); out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.alter_version); } else { const char * format_str = "Cannot execute alter data {} with version {} because another alter {} must be executed before"; - LOG_TRACE(log, format_str, entry.znode_name, entry.alter_version, head_alter); + LOG_TRACE(log, fmt::runtime(format_str), entry.znode_name, entry.alter_version, head_alter); out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.alter_version, head_alter); } @@ -1312,7 +1312,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( const char * format_str = "Not executing log entry {} of type {} for part {} " "because another DROP_RANGE or REPLACE_RANGE entry are currently executing."; - LOG_TRACE(log, format_str, entry.znode_name, entry.typeToString(), entry.new_part_name); + LOG_TRACE(log, fmt::runtime(format_str), entry.znode_name, entry.typeToString(), entry.new_part_name); out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.typeToString(), entry.new_part_name); return false; } @@ -1340,7 +1340,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( { const char * format_str = "Not executing log entry {} of type {} for part {} " "because it probably depends on {} (REPLACE_RANGE)."; - LOG_TRACE(log, format_str, entry.znode_name, entry.typeToString(), entry.new_part_name, replace_entry->znode_name); + LOG_TRACE(log, fmt::runtime(format_str), entry.znode_name, entry.typeToString(), entry.new_part_name, replace_entry->znode_name); out_postpone_reason = fmt::format(format_str, entry.znode_name, entry.typeToString(), entry.new_part_name, replace_entry->znode_name); return false; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 43cd42606a4..6cd1e2d66af 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1230,7 +1230,7 @@ bool StorageMergeTree::optimize( constexpr const char * message = "Cannot OPTIMIZE table: {}"; if (disable_reason.empty()) disable_reason = "unknown reason"; - LOG_INFO(log, message, disable_reason); + LOG_INFO(log, fmt::runtime(message), disable_reason); if (local_context->getSettingsRef().optimize_throw_if_noop) throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason); @@ -1256,7 +1256,7 @@ bool StorageMergeTree::optimize( constexpr const char * message = "Cannot OPTIMIZE table: {}"; if (disable_reason.empty()) disable_reason = "unknown reason"; - LOG_INFO(log, message, disable_reason); + LOG_INFO(log, fmt::runtime(message), disable_reason); if (local_context->getSettingsRef().optimize_throw_if_noop) throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a4b5e76c99d..9c907074531 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1211,7 +1211,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) if (unexpected_parts_nonnew_rows > 0) { - LOG_WARNING(log, sanity_report_fmt, getStorageID().getNameForLogs(), + LOG_WARNING(log, fmt::runtime(sanity_report_fmt), getStorageID().getNameForLogs(), formatReadableQuantity(unexpected_parts_rows), formatReadableQuantity(total_rows_on_filesystem), unexpected_parts.size(), unexpected_parts_rows, unexpected_parts_nonnew, unexpected_parts_nonnew_rows, parts_to_fetch.size(), parts_to_fetch_blocks); @@ -2861,17 +2861,17 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART) { /// If no one has the right part, probably not all replicas work; We will not write to log with Error level. - LOG_INFO(log, e.displayText()); + LOG_INFO(log, fmt::runtime(e.displayText())); } else if (e.code() == ErrorCodes::ABORTED) { /// Interrupted merge or downloading a part is not an error. - LOG_INFO(log, e.message()); + LOG_INFO(log, fmt::runtime(e.message())); } else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED) { /// Part cannot be added temporarily - LOG_INFO(log, e.displayText()); + LOG_INFO(log, fmt::runtime(e.displayText())); cleanup_thread.wakeup(); } else @@ -4383,7 +4383,7 @@ bool StorageReplicatedMergeTree::optimize( if (!partition_id.empty()) disable_reason += fmt::format(" (in partition {})", partition_id); String message = fmt::format(message_fmt, disable_reason); - LOG_INFO(log, message); + LOG_INFO(log, fmt::runtime(message)); return handle_noop(message); } @@ -4397,7 +4397,7 @@ bool StorageReplicatedMergeTree::optimize( if (create_result == CreateMergeEntryResult::MissingPart) { String message = "Can't create merge queue node in ZooKeeper, because some parts are missing"; - LOG_TRACE(log, message); + LOG_TRACE(log, fmt::runtime(message)); return handle_noop(message); } @@ -4410,7 +4410,7 @@ bool StorageReplicatedMergeTree::optimize( assert(try_no == max_retries); String message = fmt::format("Can't create merge queue node in ZooKeeper, because log was updated in every of {} tries", try_no); - LOG_TRACE(log, message); + LOG_TRACE(log, fmt::runtime(message)); return handle_noop(message); }; @@ -5569,7 +5569,7 @@ void StorageReplicatedMergeTree::fetchPartition( && e.code() != ErrorCodes::CANNOT_READ_ALL_DATA) throw; - LOG_INFO(log, e.displayText()); + LOG_INFO(log, fmt::runtime(e.displayText())); } return; } @@ -5706,7 +5706,7 @@ void StorageReplicatedMergeTree::fetchPartition( && e.code() != ErrorCodes::CANNOT_READ_ALL_DATA) throw; - LOG_INFO(log, e.displayText()); + LOG_INFO(log, fmt::runtime(e.displayText())); } if (!fetched)