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 <a.khuzhin@semrush.com>

v2: manual fixes
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2022-02-01 12:10:27 +03:00
parent a09bc1d72e
commit bedf208cbd
36 changed files with 82 additions and 79 deletions

View File

@ -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.

View File

@ -324,7 +324,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
}
else
{
LOG_WARNING(log, message);
LOG_WARNING(log, fmt::runtime(message));
}
}

View File

@ -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<Block> 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;
}

View File

@ -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"))

View File

@ -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"))

View File

@ -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;
}

View File

@ -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"))

View File

@ -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();
}

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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;
}

View File

@ -39,7 +39,7 @@ public:
const std::string & msg) override
{
LogsLevel db_level = static_cast<LogsLevel>(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

View File

@ -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);

View File

@ -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);

View File

@ -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));
}

View File

@ -58,15 +58,15 @@ void cassandraLogCallback(const CassLogMessage * message, void * data)
{
Poco::Logger * logger = static_cast<Poco::Logger *>(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));
}
}

View File

@ -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();

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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;

View File

@ -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);
}

View File

@ -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)

View File

@ -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); }

View File

@ -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)
{

View File

@ -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 (...)
{

View File

@ -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));
}
}

View File

@ -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<Int32>(message_type)));
LOG_ERROR(log, "Command is not supported. Command code {:d}", static_cast<Int32>(message_type));
message_transport->dropMessage();
}
}

View File

@ -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);
}
}

View File

@ -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);

View File

@ -84,7 +84,7 @@ std::pair<bool, ReplicatedMergeMutateTaskBase::PartLogWriter> 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, {}};

View File

@ -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(

View File

@ -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

View File

@ -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};
}

View File

@ -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;
}

View File

@ -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);

View File

@ -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)