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)/'

This commit is contained in:
Alexey Milovidov 2020-05-23 19:42:39 +03:00
parent 3c582fc96f
commit 8d2e80a5e2
56 changed files with 159 additions and 159 deletions

View File

@ -184,14 +184,14 @@ public:
if (sig == Signals::StopThread)
{
LOG_INFO(log, "Stop SignalListener thread");
LOG_INFO_FORMATTED(log, "Stop SignalListener thread");
break;
}
else if (sig == SIGHUP || sig == SIGUSR1)
{
LOG_DEBUG(log, "Received signal to close logs.");
LOG_DEBUG_FORMATTED(log, "Received signal to close logs.");
BaseDaemon::instance().closeLogs(BaseDaemon::instance().logger());
LOG_INFO(log, "Opened new log file after received signal.");
LOG_INFO_FORMATTED(log, "Opened new log file after received signal.");
}
else if (sig == Signals::StdTerminate)
{
@ -247,7 +247,7 @@ private:
UInt32 thread_num,
const std::string & query_id) const
{
LOG_FATAL(log, "########################################");
LOG_FATAL_FORMATTED(log, "########################################");
{
std::stringstream message;
@ -854,7 +854,7 @@ void BaseDaemon::onInterruptSignals(int signal_id)
if (sigint_signals_counter >= 2)
{
LOG_INFO(&logger(), "Received second signal Interrupt. Immediately terminate.");
LOG_INFO_FORMATTED(&logger(), "Received second signal Interrupt. Immediately terminate.");
kill();
}
}

View File

@ -241,7 +241,7 @@ void ClusterCopier::updateConfigIfNeeded()
if (!is_outdated_version && !is_expired_session)
return;
LOG_DEBUG(log, "Updating task description");
LOG_DEBUG_FORMATTED(log, "Updating task description");
reloadTaskDescription();
task_description_current_version = version_to_update;
@ -384,7 +384,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee
/// Try to make fast retries
if (num_bad_version_errors > 3)
{
LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again");
LOG_DEBUG_FORMATTED(log, "A concurrent worker has just been added, will check free worker slots again");
std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution<int>(1, 1000)(task_cluster->random_engine));
std::this_thread::sleep_for(random_sleep_time);
num_bad_version_errors = 0;
@ -854,7 +854,7 @@ bool ClusterCopier::tryDropPartitionPiece(
}
else
{
LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing");
LOG_DEBUG_FORMATTED(log, "Clean state is altered when dropping the partition, cowardly bailing");
/// clean state is stale
return false;
}
@ -1165,17 +1165,17 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
auto create_is_dirty_node = [&] (const CleanStateClock & clock)
{
if (clock.is_stale())
LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing");
LOG_DEBUG_FORMATTED(log, "Clean state clock is stale while setting dirty flag, cowardly bailing");
else if (!clock.is_clean())
LOG_DEBUG(log, "Thank you, Captain Obvious");
LOG_DEBUG_FORMATTED(log, "Thank you, Captain Obvious");
else if (clock.discovery_version)
{
LOG_DEBUG(log, "Updating clean state clock");
LOG_DEBUG_FORMATTED(log, "Updating clean state clock");
zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value());
}
else
{
LOG_DEBUG(log, "Creating clean state clock");
LOG_DEBUG_FORMATTED(log, "Creating clean state clock");
zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent);
}
};
@ -1571,7 +1571,7 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na
void ClusterCopier::dropHelpingTables(const TaskTable & task_table)
{
LOG_DEBUG(log, "Removing helping tables");
LOG_DEBUG_FORMATTED(log, "Removing helping tables");
for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number)
{
DatabaseAndTableName original_table = task_table.table_push;
@ -1598,7 +1598,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table)
void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name)
{
LOG_DEBUG(log, "Try drop partition partition from all helping tables.");
LOG_DEBUG_FORMATTED(log, "Try drop partition partition from all helping tables.");
for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number)
{
DatabaseAndTableName original_table = task_table.table_push;

View File

@ -216,7 +216,7 @@ try
attachSystemTables();
loadMetadata(*context);
DatabaseCatalog::instance().loadDatabases();
LOG_DEBUG(log, "Loaded metadata.");
LOG_DEBUG_FORMATTED(log, "Loaded metadata.");
}
else
{

View File

@ -165,7 +165,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
if (is_help)
return Application::EXIT_OK;
LOG_INFO(log, "Starting up");
LOG_INFO_FORMATTED(log, "Starting up");
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, hostname, port, log);
socket.setReceiveTimeout(http_timeout);
@ -191,8 +191,8 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, "Listening http://" + address.toString());
SCOPE_EXIT({
LOG_DEBUG(log, "Received termination signal.");
LOG_DEBUG(log, "Waiting for current connections to close.");
LOG_DEBUG_FORMATTED(log, "Received termination signal.");
LOG_DEBUG_FORMATTED(log, "Waiting for current connections to close.");
server.stop();
for (size_t count : ext::range(1, 6))
{

View File

@ -705,7 +705,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
}
processQuery(request, params, response, used_output);
LOG_INFO(log, "Done processing query");
LOG_INFO_FORMATTED(log, "Done processing query");
}
catch (...)
{

View File

@ -96,7 +96,7 @@ void InterserverIOHTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & requ
if (auto [message, success] = checkAuthentication(request); success)
{
processQuery(request, response, used_output);
LOG_INFO(log, "Done processing query");
LOG_INFO_FORMATTED(log, "Done processing query");
}
else
{

View File

@ -73,7 +73,7 @@ void MySQLHandler::run()
Handshake handshake(server_capability_flags, connection_id, VERSION_STRING + String("-") + VERSION_NAME, auth_plugin->getName(), auth_plugin->getAuthPluginData());
packet_sender->sendPacket<Handshake>(handshake, true);
LOG_TRACE(log, "Sent handshake");
LOG_TRACE_FORMATTED(log, "Sent handshake");
HandshakeResponse handshake_response;
finishHandshake(handshake_response);

View File

@ -100,7 +100,7 @@ void MySQLHandlerFactory::readRSAKeys()
void MySQLHandlerFactory::generateRSAKeys()
{
LOG_TRACE(log, "Generating new RSA key pair.");
LOG_TRACE_FORMATTED(log, "Generating new RSA key pair.");
public_key.reset(RSA_new());
if (!public_key)
throw Exception("Failed to allocate RSA key. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);

View File

@ -96,7 +96,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
}
catch (...)
{
LOG_ERROR((&Logger::get("ReplicasStatusHandler")), "Cannot send exception to client");
LOG_ERROR_FORMATTED((&Logger::get("ReplicasStatusHandler")), "Cannot send exception to client");
}
}
}

View File

@ -234,7 +234,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());
if (ThreadFuzzer::instance().isEffective())
LOG_WARNING(log, "ThreadFuzzer is enabled. Application will run slowly and unstable.");
LOG_WARNING_FORMATTED(log, "ThreadFuzzer is enabled. Application will run slowly and unstable.");
/** Context contains all that query execution is dependent:
* settings, available functions, data types, aggregate functions, databases...
@ -274,11 +274,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
if (hasLinuxCapability(CAP_IPC_LOCK))
{
LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
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));
else
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed");
LOG_TRACE_FORMATTED(log, "The memory map of clickhouse executable has been mlock'ed");
}
else
{
@ -326,11 +326,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
* It is important to do early, not in destructor of Context, because
* table engines could use Context on destroy.
*/
LOG_INFO(log, "Shutting down storages.");
LOG_INFO_FORMATTED(log, "Shutting down storages.");
global_context->shutdown();
LOG_DEBUG(log, "Shut down storages.");
LOG_DEBUG_FORMATTED(log, "Shut down storages.");
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
* At this moment, no one could own shared part of Context.
@ -338,7 +338,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context_ptr = nullptr;
global_context.reset();
shared_context.reset();
LOG_DEBUG(log, "Destroyed global context.");
LOG_DEBUG_FORMATTED(log, "Destroyed global context.");
});
/// Try to increase limit on number of open files.
@ -370,7 +370,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
Poco::ErrorHandler::set(&error_handler);
/// Initialize DateLUT early, to not interfere with running time of first query.
LOG_DEBUG(log, "Initializing DateLUT.");
LOG_DEBUG_FORMATTED(log, "Initializing DateLUT.");
DateLUT::instance();
LOG_TRACE(log, "Initialized DateLUT with time zone '" << DateLUT::instance().getTimeZone() << "'.");
@ -550,7 +550,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Size of cache for marks (index of MergeTree family of tables). It is mandatory.
size_t mark_cache_size = config().getUInt64("mark_cache_size");
if (!mark_cache_size)
LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation.");
LOG_ERROR_FORMATTED(log, "Too low mark cache size will lead to severe performance degradation.");
if (mark_cache_size > max_cache_size)
{
mark_cache_size = max_cache_size;
@ -610,7 +610,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
tryLogCurrentException(log, "Caught exception while loading metadata");
throw;
}
LOG_DEBUG(log, "Loaded metadata.");
LOG_DEBUG_FORMATTED(log, "Loaded metadata.");
/// Init trace collector only after trace_log system table was created
/// Disable it if we collect test coverage information, because it will work extremely slow.
@ -650,11 +650,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Describe multiple reasons when query profiler cannot work.
#if !USE_UNWIND
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work without bundled unwind (stack unwinding) library.");
LOG_INFO_FORMATTED(log, "Query Profiler and TraceCollector are disabled because they cannot work without bundled unwind (stack unwinding) library.");
#endif
#if WITH_COVERAGE
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage.");
LOG_INFO_FORMATTED(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage.");
#endif
#if defined(SANITIZER)
@ -707,7 +707,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
" Note that it will not work on 'nosuid' mounted filesystems.");
}
#else
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
LOG_INFO_FORMATTED(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
#endif
{
@ -974,11 +974,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, message.str());
}
LOG_INFO(log, "Ready for connections.");
LOG_INFO_FORMATTED(log, "Ready for connections.");
SCOPE_EXIT({
LOG_DEBUG(log, "Received termination signal.");
LOG_DEBUG(log, "Waiting for current connections to close.");
LOG_DEBUG_FORMATTED(log, "Received termination signal.");
LOG_DEBUG_FORMATTED(log, "Waiting for current connections to close.");
is_cancelled = true;
@ -1030,7 +1030,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Dump coverage here, because std::atexit callback would not be called.
dumpCoverageReportIfPossible();
LOG_INFO(log, "Will shutdown forcefully.");
LOG_INFO_FORMATTED(log, "Will shutdown forcefully.");
_exit(Application::EXIT_OK);
}
});
@ -1051,7 +1051,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
catch (...)
{
LOG_ERROR(log, "Caught exception while loading dictionaries.");
LOG_ERROR_FORMATTED(log, "Caught exception while loading dictionaries.");
throw;
}

View File

@ -76,7 +76,7 @@ void TCPHandler::runImpl()
if (in->eof())
{
LOG_WARNING(log, "Client has not sent any data.");
LOG_WARNING_FORMATTED(log, "Client has not sent any data.");
return;
}
@ -89,13 +89,13 @@ void TCPHandler::runImpl()
{
if (e.code() == ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT)
{
LOG_DEBUG(log, "Client has connected to wrong port.");
LOG_DEBUG_FORMATTED(log, "Client has connected to wrong port.");
return;
}
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
{
LOG_WARNING(log, "Client has gone away.");
LOG_WARNING_FORMATTED(log, "Client has gone away.");
return;
}
@ -140,7 +140,7 @@ void TCPHandler::runImpl()
{
if (idle_time.elapsedSeconds() > connection_settings.idle_connection_timeout)
{
LOG_TRACE(log, "Closing idle connection");
LOG_TRACE_FORMATTED(log, "Closing idle connection");
return;
}
}
@ -345,7 +345,7 @@ void TCPHandler::runImpl()
{
/** Could not send exception information to the client. */
network_error = true;
LOG_WARNING(log, "Client has gone away.");
LOG_WARNING_FORMATTED(log, "Client has gone away.");
}
try
@ -356,7 +356,7 @@ void TCPHandler::runImpl()
catch (...)
{
network_error = true;
LOG_WARNING(log, "Can't read external tables after query failure.");
LOG_WARNING_FORMATTED(log, "Can't read external tables after query failure.");
}
@ -1080,7 +1080,7 @@ bool TCPHandler::isQueryCancelled()
case Protocol::Client::Cancel:
if (state.empty())
throw NetException("Unexpected packet Cancel received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
LOG_INFO(log, "Query was cancelled.");
LOG_INFO_FORMATTED(log, "Query was cancelled.");
state.is_cancelled = true;
return true;
@ -1198,7 +1198,7 @@ void TCPHandler::run()
{
runImpl();
LOG_INFO(log, "Done processing connection.");
LOG_INFO_FORMATTED(log, "Done processing connection.");
}
catch (Poco::Exception & e)
{

View File

@ -40,7 +40,7 @@ public:
}
catch (const Poco::Net::NetException &)
{
LOG_TRACE(log, "TCP Request. Client is not connected (most likely RST packet was sent).");
LOG_TRACE_FORMATTED(log, "TCP Request. Client is not connected (most likely RST packet was sent).");
return new DummyTCPHandler(socket);
}
}

View File

@ -794,6 +794,6 @@ void AccessRights::logTree() const
if (root)
root->logTree(log);
else
LOG_TRACE(log, "Tree: NULL");
LOG_TRACE_FORMATTED(log, "Tree: NULL");
}
}

View File

@ -287,14 +287,14 @@ void Connection::forceConnected(const ConnectionTimeouts & timeouts)
}
else if (!ping())
{
LOG_TRACE(log_wrapper.get(), "Connection was closed, will reconnect.");
LOG_TRACE_FORMATTED(log_wrapper.get(), "Connection was closed, will reconnect.");
connect(timeouts);
}
}
bool Connection::ping()
{
// LOG_TRACE(log_wrapper.get(), "Ping");
// LOG_TRACE_FORMATTED(log_wrapper.get(), "Ping");
TimeoutSetter timeout_setter(*socket, sync_request_timeout, true);
try
@ -646,7 +646,7 @@ std::optional<UInt64> Connection::checkPacket(size_t timeout_microseconds)
if (hasReadPendingData() || poll(timeout_microseconds))
{
// LOG_TRACE(log_wrapper.get(), "Receiving packet type");
// LOG_TRACE_FORMATTED(log_wrapper.get(), "Receiving packet type");
UInt64 packet_type;
readVarUInt(packet_type, *in);

View File

@ -306,7 +306,7 @@ private:
auto it = cells.find(key);
if (it == cells.end())
{
LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
LOG_ERROR_FORMATTED(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
abort();
}
@ -324,7 +324,7 @@ private:
if (current_size > (1ull << 63))
{
LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
LOG_ERROR_FORMATTED(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
abort();
}
}

View File

@ -122,7 +122,7 @@ public:
return Entry(*items.back());
}
LOG_INFO(log, "No free connections in pool. Waiting.");
LOG_INFO_FORMATTED(log, "No free connections in pool. Waiting.");
if (timeout < 0)
available.wait(lock);

View File

@ -48,7 +48,7 @@ TraceCollector::TraceCollector(std::shared_ptr<TraceLog> trace_log_)
TraceCollector::~TraceCollector()
{
if (!thread.joinable())
LOG_ERROR(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined");
LOG_ERROR_FORMATTED(&Poco::Logger::get("TraceCollector"), "TraceCollector thread is malformed and cannot be joined");
else
stop();

View File

@ -16,7 +16,7 @@ public:
size_t get()
{
LOG_TRACE(log, "Get increment");
LOG_TRACE_FORMATTED(log, "Get increment");
size_t result = 0;
std::string result_str;

View File

@ -34,7 +34,7 @@ bool ZooKeeperHolder::replaceZooKeeperSessionToNewOne()
}
else
{
LOG_ERROR(log, "replaceZooKeeperSessionToNewOne(): Fail to replace zookeeper session to new one because handlers for old zookeeper session still exists.");
LOG_ERROR_FORMATTED(log, "replaceZooKeeperSessionToNewOne(): Fail to replace zookeeper session to new one because handlers for old zookeeper session still exists.");
return false;
}
}

View File

@ -179,7 +179,7 @@ BackgroundSchedulePool::~BackgroundSchedulePool()
queue.wakeUpAll();
delayed_thread.join();
LOG_TRACE(&Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish.");
LOG_TRACE_FORMATTED(&Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish.");
for (auto & thread : threads)
thread.join();
}

View File

@ -1044,17 +1044,17 @@ public:
AuthSwitchResponse response;
packet_sender->receivePacket(response);
auth_response = response.value;
LOG_TRACE(log, "Authentication method mismatch.");
LOG_TRACE_FORMATTED(log, "Authentication method mismatch.");
}
else
{
LOG_TRACE(log, "Authentication method match.");
LOG_TRACE_FORMATTED(log, "Authentication method match.");
}
bool sent_public_key = false;
if (auth_response == "\1")
{
LOG_TRACE(log, "Client requests public key.");
LOG_TRACE_FORMATTED(log, "Client requests public key.");
BIO * mem = BIO_new(BIO_s_mem());
SCOPE_EXIT(BIO_free(mem));
if (PEM_write_bio_RSA_PUBKEY(mem, &public_key) != 1)
@ -1080,7 +1080,7 @@ public:
}
else
{
LOG_TRACE(log, "Client didn't request public key.");
LOG_TRACE_FORMATTED(log, "Client didn't request public key.");
}
String password;
@ -1092,7 +1092,7 @@ public:
*/
if (!is_secure_connection && !auth_response->empty() && auth_response != String("\0", 1))
{
LOG_TRACE(log, "Received nonempty password.");
LOG_TRACE_FORMATTED(log, "Received nonempty password.");
auto ciphertext = reinterpret_cast<unsigned char *>(auth_response->data());
unsigned char plaintext[RSA_size(&private_key)];
@ -1100,7 +1100,7 @@ public:
if (plaintext_size == -1)
{
if (!sent_public_key)
LOG_WARNING(log, "Client could have encrypted password with different public key since it didn't request it from server.");
LOG_WARNING_FORMATTED(log, "Client could have encrypted password with different public key since it didn't request it from server.");
throw Exception("Failed to decrypt auth data. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
}
@ -1116,7 +1116,7 @@ public:
}
else
{
LOG_TRACE(log, "Received empty password");
LOG_TRACE_FORMATTED(log, "Received empty password");
}
if (!password.empty() && password.back() == 0)

View File

@ -113,7 +113,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
{
if (isCancelled())
{
LOG_DEBUG(log, "Query was cancelled during set / join or temporary table creation.");
LOG_DEBUG_FORMATTED(log, "Query was cancelled during set / join or temporary table creation.");
return;
}
@ -181,7 +181,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
}
else
{
LOG_DEBUG(log, "Subquery has empty result.");
LOG_DEBUG_FORMATTED(log, "Subquery has empty result.");
}
}

View File

@ -292,12 +292,12 @@ void MergingAggregatedMemoryEfficientBlockInputStream::finalize()
if (!started)
return;
LOG_TRACE(log, "Waiting for threads to finish");
LOG_TRACE_FORMATTED(log, "Waiting for threads to finish");
if (parallel_merge_data)
parallel_merge_data->pool.wait();
LOG_TRACE(log, "Waited for threads to finish");
LOG_TRACE_FORMATTED(log, "Waited for threads to finish");
}
@ -555,7 +555,7 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate
/// Not yet partitioned (splitted to buckets) block. Will partition it and place result to 'splitted_blocks'.
if (input.block.info.bucket_num == -1 && input.block && input.splitted_blocks.empty())
{
LOG_TRACE(&Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split.");
LOG_TRACE_FORMATTED(&Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split.");
input.splitted_blocks = aggregator.convertBlockToTwoLevel(input.block);
input.block = Block();

View File

@ -157,7 +157,7 @@ void ParallelAggregatingBlockInputStream::execute()
for (size_t i = 0; i < max_threads; ++i)
threads_data.emplace_back(keys_size, aggregates_size);
LOG_TRACE(log, "Aggregating");
LOG_TRACE_FORMATTED(log, "Aggregating");
Stopwatch watch;

View File

@ -107,7 +107,7 @@ protected:
if (!started)
return;
LOG_TRACE(log, "Waiting for threads to finish");
LOG_TRACE_FORMATTED(log, "Waiting for threads to finish");
std::exception_ptr exception;
if (!all_read)
@ -137,7 +137,7 @@ protected:
processor.wait();
LOG_TRACE(log, "Waited for threads to finish");
LOG_TRACE_FORMATTED(log, "Waited for threads to finish");
if (exception)
std::rethrow_exception(exception);

View File

@ -194,7 +194,7 @@ void DatabaseOrdinary::loadStoredObjects(
void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
{
LOG_INFO(log, "Starting up tables.");
LOG_INFO_FORMATTED(log, "Starting up tables.");
const size_t total_tables = tables.size();
if (!total_tables)

View File

@ -202,7 +202,7 @@ BlockInputStreamPtr ClickHouseDictionarySource::createStreamForSelectiveLoad(con
std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const
{
LOG_TRACE(log, "Performing invalidate query");
LOG_TRACE_FORMATTED(log, "Performing invalidate query");
if (is_local)
{
Context query_context = context;

View File

@ -8,7 +8,7 @@ RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_p
{
Logger * log = &Logger::get("RegionsHierarchies");
LOG_DEBUG(log, "Adding default regions hierarchy");
LOG_DEBUG_FORMATTED(log, "Adding default regions hierarchy");
data.emplace("", data_provider->getDefaultHierarchySource());
for (const auto & name : data_provider->listCustomHierarchies())

View File

@ -28,7 +28,7 @@ void RegionsHierarchy::reload()
if (!data_source->isModified())
return;
LOG_DEBUG(log, "Reloading regions hierarchy");
LOG_DEBUG_FORMATTED(log, "Reloading regions hierarchy");
const size_t initial_size = 10000;
const size_t max_size = 15000000;

View File

@ -43,7 +43,7 @@ std::string RegionsNames::dumpSupportedLanguagesNames()
void RegionsNames::reload()
{
Logger * log = &Logger::get("RegionsNames");
LOG_DEBUG(log, "Reloading regions names");
LOG_DEBUG_FORMATTED(log, "Reloading regions names");
RegionID max_region_id = 0;
for (size_t language_id = 0; language_id < total_languages; ++language_id)

View File

@ -241,10 +241,10 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry &
}
if (0 == fetched_rows)
LOG_ERROR(log, "Cannot find table in SHOW TABLE STATUS result.");
LOG_ERROR_FORMATTED(log, "Cannot find table in SHOW TABLE STATUS result.");
if (fetched_rows > 1)
LOG_ERROR(log, "Found more than one table in SHOW TABLE STATUS result.");
LOG_ERROR_FORMATTED(log, "Found more than one table in SHOW TABLE STATUS result.");
}
catch (...)
{

View File

@ -115,7 +115,7 @@ void WriteBufferFromS3::writePart(const String & data)
if (part_tags.size() == S3_WARN_MAX_PARTS)
{
// Don't throw exception here by ourselves but leave the decision to take by S3 server.
LOG_WARNING(log, "Maximum part number in S3 protocol has reached (too many parts). Server may not accept this whole upload.");
LOG_WARNING_FORMATTED(log, "Maximum part number in S3 protocol has reached (too many parts). Server may not accept this whole upload.");
}
Aws::S3::Model::UploadPartRequest req;

View File

@ -75,7 +75,7 @@ AggregatedDataVariants::~AggregatedDataVariants()
void AggregatedDataVariants::convertToTwoLevel()
{
if (aggregator)
LOG_TRACE(aggregator->log, "Converting aggregation data to two-level.");
LOG_TRACE_FORMATTED(aggregator->log, "Converting aggregation data to two-level.");
switch (type)
{
@ -912,7 +912,7 @@ void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVaria
*/
bool no_more_keys = false;
LOG_TRACE(log, "Aggregating");
LOG_TRACE_FORMATTED(log, "Aggregating");
Stopwatch watch;
@ -1255,7 +1255,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
if (isCancelled())
return BlocksList();
LOG_TRACE(log, "Converting aggregated data to blocks");
LOG_TRACE_FORMATTED(log, "Converting aggregated data to blocks");
Stopwatch watch;
@ -1562,7 +1562,7 @@ public:
~MergingAndConvertingBlockInputStream() override
{
LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish");
LOG_TRACE_FORMATTED(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish");
/// We need to wait for threads to finish before destructor of 'parallel_merge_data',
/// because the threads access 'parallel_merge_data'.
@ -1737,7 +1737,7 @@ ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedData
if (data_variants.empty())
throw Exception("Empty data passed to Aggregator::mergeAndConvertToBlocks.", ErrorCodes::EMPTY_DATA_PASSED);
LOG_TRACE(log, "Merging aggregated data");
LOG_TRACE_FORMATTED(log, "Merging aggregated data");
ManyAggregatedDataVariants non_empty_data;
non_empty_data.reserve(data_variants.size());
@ -1929,7 +1929,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
BucketToBlocks bucket_to_blocks;
/// Read all the data.
LOG_TRACE(log, "Reading blocks of partially aggregated data.");
LOG_TRACE_FORMATTED(log, "Reading blocks of partially aggregated data.");
size_t total_input_rows = 0;
size_t total_input_blocks = 0;
@ -1997,7 +1997,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari
* That is, the keys in the end can be significantly larger than max_rows_to_group_by.
*/
LOG_TRACE(log, "Merging partially aggregated two-level data.");
LOG_TRACE_FORMATTED(log, "Merging partially aggregated two-level data.");
auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupStatusPtr thread_group)
{
@ -2046,7 +2046,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari
if (thread_pool)
thread_pool->wait();
LOG_TRACE(log, "Merged partially aggregated two-level data.");
LOG_TRACE_FORMATTED(log, "Merged partially aggregated two-level data.");
}
if (isCancelled())
@ -2057,7 +2057,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari
if (has_blocks_with_unknown_bucket)
{
LOG_TRACE(log, "Merging partially aggregated single-level data.");
LOG_TRACE_FORMATTED(log, "Merging partially aggregated single-level data.");
bool no_more_keys = false;
@ -2086,7 +2086,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
LOG_TRACE(log, "Merged partially aggregated single-level data.");
LOG_TRACE_FORMATTED(log, "Merged partially aggregated single-level data.");
}
}
@ -2347,7 +2347,7 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result)
if (result.empty())
return;
LOG_TRACE(log, "Destroying aggregate states");
LOG_TRACE_FORMATTED(log, "Destroying aggregate states");
/// In what data structure is the data aggregated?
if (result.type == AggregatedDataVariants::Type::without_key || params.overflow_row)

View File

@ -364,7 +364,7 @@ static void filterAndSortQueueNodes(Strings & all_nodes)
void DDLWorker::processTasks()
{
LOG_DEBUG(log, "Processing tasks");
LOG_DEBUG_FORMATTED(log, "Processing tasks");
auto zookeeper = tryGetZooKeeper();
Strings queue_nodes = zookeeper->getChildren(queue_dir, nullptr, queue_updated_event);
@ -823,7 +823,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zookeeper)
{
LOG_DEBUG(log, "Cleaning queue");
LOG_DEBUG_FORMATTED(log, "Cleaning queue");
Strings queue_nodes = zookeeper->getChildren(queue_dir);
filterAndSortQueueNodes(queue_nodes);
@ -958,7 +958,7 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry)
void DDLWorker::runMainThread()
{
setThreadName("DDLWorker");
LOG_DEBUG(log, "Started DDLWorker thread");
LOG_DEBUG_FORMATTED(log, "Started DDLWorker thread");
bool initialized = false;
do
@ -996,7 +996,7 @@ void DDLWorker::runMainThread()
cleanup_event->set();
processTasks();
LOG_DEBUG(log, "Waiting a watch");
LOG_DEBUG_FORMATTED(log, "Waiting a watch");
queue_updated_event->wait();
}
catch (const Coordination::Exception & e)
@ -1043,7 +1043,7 @@ void DDLWorker::runMainThread()
void DDLWorker::runCleanupThread()
{
setThreadName("DDLWorkerClnr");
LOG_DEBUG(log, "Started DDLWorker cleanup thread");
LOG_DEBUG_FORMATTED(log, "Started DDLWorker cleanup thread");
Int64 last_cleanup_time_seconds = 0;
while (!stop_flag)
@ -1057,7 +1057,7 @@ void DDLWorker::runCleanupThread()
Int64 current_time_seconds = Poco::Timestamp().epochTime();
if (last_cleanup_time_seconds && current_time_seconds < last_cleanup_time_seconds + cleanup_delay_period)
{
LOG_TRACE(log, "Too early to clean queue, will do it later.");
LOG_TRACE_FORMATTED(log, "Too early to clean queue, will do it later.");
continue;
}

View File

@ -68,7 +68,7 @@ bool EmbeddedDictionaries::reloadImpl(const bool throw_on_error, const bool forc
* An attempt is made to load each directory separately.
*/
LOG_INFO(log, "Loading dictionaries.");
LOG_INFO_FORMATTED(log, "Loading dictionaries.");
bool was_exception = false;
@ -89,7 +89,7 @@ bool EmbeddedDictionaries::reloadImpl(const bool throw_on_error, const bool forc
was_exception = true;
if (!was_exception)
LOG_INFO(log, "Loaded dictionaries.");
LOG_INFO_FORMATTED(log, "Loaded dictionaries.");
return !was_exception;
}

View File

@ -43,7 +43,7 @@ void InternalTextLogsQueue::pushBlock(Block && log_block)
if (blocksHaveEqualStructure(sample_block, log_block))
emplace(log_block.mutateColumns());
else
LOG_WARNING(&Poco::Logger::get("InternalTextLogsQueue"), "Log block have different structure");
LOG_WARNING_FORMATTED(&Poco::Logger::get("InternalTextLogsQueue"), "Log block have different structure");
}
const char * InternalTextLogsQueue::getPriorityName(int priority)

View File

@ -398,7 +398,7 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery &)
if (auto * storage_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get()))
{
LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for it to become empty");
LOG_TRACE_FORMATTED(log, "Synchronizing entries in replica's queue with table's log and waiting for it to become empty");
if (!storage_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.totalMilliseconds()))
{
LOG_ERROR(log, "SYNC REPLICA " + table_id.getNameForLogs() + ": Timed out!");

View File

@ -231,7 +231,7 @@ ProcessListEntry::~ProcessListEntry()
auto user_process_list_it = parent.user_to_queries.find(user);
if (user_process_list_it == parent.user_to_queries.end())
{
LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList");
LOG_ERROR_FORMATTED(&Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList");
std::terminate();
}
@ -250,7 +250,7 @@ ProcessListEntry::~ProcessListEntry()
if (!found)
{
LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
LOG_ERROR_FORMATTED(&Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
std::terminate();
}
parent.have_space.notify_all();

View File

@ -377,7 +377,7 @@ void SystemLog<LogElement>::savingThreadFunction()
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
LOG_TRACE(log, "Terminating");
LOG_TRACE_FORMATTED(log, "Terminating");
}
@ -426,7 +426,7 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
flush_event.notify_all();
}
LOG_TRACE(log, "Flushed system log");
LOG_TRACE_FORMATTED(log, "Flushed system log");
}

View File

@ -218,7 +218,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
#if defined(__linux__)
if (os_thread_priority)
{
LOG_TRACE(log, "Resetting nice");
LOG_TRACE_FORMATTED(log, "Resetting nice");
if (0 != setpriority(PRIO_PROCESS, thread_id, 0))
LOG_ERROR(log, "Cannot 'setpriority' back to zero: " << errnoToString(ErrorCodes::CANNOT_SET_THREAD_PRIORITY, errno));

View File

@ -515,7 +515,7 @@ void AggregatingTransform::consume(Chunk chunk)
if (!is_consume_started)
{
LOG_TRACE(log, "Aggregating");
LOG_TRACE_FORMATTED(log, "Aggregating");
is_consume_started = true;
}

View File

@ -103,7 +103,7 @@ void CreatingSetsTransform::finishSubquery(SubqueryForSet & subquery)
}
else
{
LOG_DEBUG(log, "Subquery has empty result.");
LOG_DEBUG_FORMATTED(log, "Subquery has empty result.");
}
}

View File

@ -20,7 +20,7 @@ void MergingAggregatedTransform::consume(Chunk chunk)
if (!consume_started)
{
consume_started = true;
LOG_TRACE(log, "Reading blocks of partially aggregated data.");
LOG_TRACE_FORMATTED(log, "Reading blocks of partially aggregated data.");
}
total_input_rows += chunk.getNumRows();

View File

@ -153,7 +153,7 @@ void StorageDistributedDirectoryMonitor::run()
}
else
{
LOG_DEBUG(log, "Skipping send data over distributed table.");
LOG_DEBUG_FORMATTED(log, "Skipping send data over distributed table.");
}
const auto now = std::chrono::system_clock::now();

View File

@ -163,7 +163,7 @@ void ReadBufferFromKafkaConsumer::commit()
}
else
{
LOG_TRACE(log, "Nothing to commit.");
LOG_TRACE_FORMATTED(log, "Nothing to commit.");
}
offsets_stored = 0;
@ -204,7 +204,7 @@ void ReadBufferFromKafkaConsumer::subscribe()
void ReadBufferFromKafkaConsumer::unsubscribe()
{
LOG_TRACE(log, "Re-joining claimed consumer after failure");
LOG_TRACE_FORMATTED(log, "Re-joining claimed consumer after failure");
messages.clear();
current = messages.begin();
@ -239,7 +239,7 @@ void ReadBufferFromKafkaConsumer::resetToLastCommitted(const char * msg)
{
if (assignment.empty())
{
LOG_TRACE(log, "Not assignned. Can't reset to last committed position.");
LOG_TRACE_FORMATTED(log, "Not assignned. Can't reset to last committed position.");
return;
}
auto committed_offset = consumer->get_offsets_committed(consumer->get_assignment());
@ -312,7 +312,7 @@ bool ReadBufferFromKafkaConsumer::nextImpl()
}
else
{
LOG_TRACE(log, "Can't get assignment");
LOG_TRACE_FORMATTED(log, "Can't get assignment");
stalled = true;
return false;
}
@ -320,7 +320,7 @@ bool ReadBufferFromKafkaConsumer::nextImpl()
}
else
{
LOG_TRACE(log, "Stalled");
LOG_TRACE_FORMATTED(log, "Stalled");
stalled = true;
return false;
}

View File

@ -212,7 +212,7 @@ void StorageKafka::shutdown()
// Interrupt streaming thread
stream_cancelled = true;
LOG_TRACE(log, "Waiting for cleanup");
LOG_TRACE_FORMATTED(log, "Waiting for cleanup");
task->deactivate();
// Close all consumers
@ -340,12 +340,12 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & conf)
status = rd_kafka_conf_interceptor_add_on_new(conf.get_handle(), "setThreadName", rdKafkaOnNew, self);
if (status != RD_KAFKA_RESP_ERR_NO_ERROR)
LOG_ERROR(log, "Cannot set new interceptor");
LOG_ERROR_FORMATTED(log, "Cannot set new interceptor");
// cppkafka always copy the configuration
status = rd_kafka_conf_interceptor_add_on_conf_dup(conf.get_handle(), "setThreadName", rdKafkaOnConfDup, self);
if (status != RD_KAFKA_RESP_ERR_NO_ERROR)
LOG_ERROR(log, "Cannot set dup conf interceptor");
LOG_ERROR_FORMATTED(log, "Cannot set dup conf interceptor");
}
}

View File

@ -851,7 +851,7 @@ Int64 MergeTreeData::getMaxBlockNumber() const
void MergeTreeData::loadDataParts(bool skip_sanity_checks)
{
LOG_DEBUG(log, "Loading data parts");
LOG_DEBUG_FORMATTED(log, "Loading data parts");
const auto settings = getSettings();
std::vector<std::pair<String, DiskPtr>> part_names_with_disks;
@ -901,7 +901,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (part_names_with_disks.empty())
{
LOG_DEBUG(log, "There is no data parts");
LOG_DEBUG_FORMATTED(log, "There is no data parts");
return;
}
@ -1323,11 +1323,11 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_
void MergeTreeData::dropAllData()
{
LOG_TRACE(log, "dropAllData: waiting for locks.");
LOG_TRACE_FORMATTED(log, "dropAllData: waiting for locks.");
auto lock = lockParts();
LOG_TRACE(log, "dropAllData: removing data from memory.");
LOG_TRACE_FORMATTED(log, "dropAllData: removing data from memory.");
DataPartsVector all_parts(data_parts_by_info.begin(), data_parts_by_info.end());
@ -1336,7 +1336,7 @@ void MergeTreeData::dropAllData()
global_context.dropCaches();
LOG_TRACE(log, "dropAllData: removing data from filesystem.");
LOG_TRACE_FORMATTED(log, "dropAllData: removing data from filesystem.");
/// Removing of each data part before recursive removal of directory is to speed-up removal, because there will be less number of syscalls.
clearPartsFromFilesystem(all_parts);
@ -1344,7 +1344,7 @@ void MergeTreeData::dropAllData()
for (const auto & [path, disk] : getRelativeDataPathsWithDisks())
disk->removeRecursive(path);
LOG_TRACE(log, "dropAllData: done.");
LOG_TRACE_FORMATTED(log, "dropAllData: done.");
}
namespace
@ -2844,7 +2844,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
renamed_parts.tryRenameAll();
/// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
LOG_DEBUG(log, "Checking parts");
LOG_DEBUG_FORMATTED(log, "Checking parts");
MutableDataPartsVector loaded_parts;
loaded_parts.reserve(renamed_parts.old_and_new_names.size());
for (const auto & part_names : renamed_parts.old_and_new_names)

View File

@ -694,7 +694,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
total_size += part->getBytesOnDisk();
if (total_size >= data_settings->min_merge_bytes_to_use_direct_io)
{
LOG_DEBUG(log, "Will merge parts reading files in O_DIRECT");
LOG_DEBUG_FORMATTED(log, "Will merge parts reading files in O_DIRECT");
read_with_direct_io = true;
break;

View File

@ -530,7 +530,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
if (no_data)
{
LOG_DEBUG(log, "Sampling yields no data.");
LOG_DEBUG_FORMATTED(log, "Sampling yields no data.");
return {};
}

View File

@ -367,7 +367,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
if (quorum)
{
/// We are waiting for quorum to be satisfied.
LOG_TRACE(log, "Waiting for quorum");
LOG_TRACE_FORMATTED(log, "Waiting for quorum");
String quorum_status_path = storage.zookeeper_path + "/quorum/status";
@ -406,7 +406,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
ErrorCodes::UNKNOWN_STATUS_OF_INSERT);
}
LOG_TRACE(log, "Quorum satisfied");
LOG_TRACE_FORMATTED(log, "Quorum satisfied");
}
}

View File

@ -243,7 +243,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na
if (need_stop)
{
LOG_INFO(log, "Checking part was cancelled.");
LOG_INFO_FORMATTED(log, "Checking part was cancelled.");
return {part_name, false, "Checking part was cancelled"};
}
@ -325,7 +325,7 @@ void ReplicatedMergeTreePartCheckThread::run()
{
if (!parts_set.empty())
{
LOG_ERROR(log, "Non-empty parts_set with empty parts_queue. This is a bug.");
LOG_ERROR_FORMATTED(log, "Non-empty parts_set with empty parts_queue. This is a bug.");
parts_set.clear();
}
}
@ -359,7 +359,7 @@ void ReplicatedMergeTreePartCheckThread::run()
if (parts_queue.empty())
{
LOG_ERROR(log, "Someone erased cheking part from parts_queue. This is a bug.");
LOG_ERROR_FORMATTED(log, "Someone erased cheking part from parts_queue. This is a bug.");
}
else
{

View File

@ -106,7 +106,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, {});
LOG_TRACE(log, "Loaded queue");
LOG_TRACE_FORMATTED(log, "Loaded queue");
return updated;
}

View File

@ -68,11 +68,11 @@ void ReplicatedMergeTreeRestartingThread::run()
if (first_time)
{
LOG_DEBUG(log, "Activating replica.");
LOG_DEBUG_FORMATTED(log, "Activating replica.");
}
else
{
LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
LOG_WARNING_FORMATTED(log, "ZooKeeper session has expired. Switching to a new session.");
bool old_val = false;
if (storage.is_readonly.compare_exchange_strong(old_val, true))
@ -189,7 +189,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
if (storage_settings->replicated_can_become_leader)
storage.enterLeaderElection();
else
LOG_INFO(log, "Will not enter leader election because replicated_can_become_leader=0");
LOG_INFO_FORMATTED(log, "Will not enter leader election because replicated_can_become_leader=0");
/// Anything above can throw a KeeperException if something is wrong with ZK.
/// Anything below should not throw exceptions.
@ -337,7 +337,7 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown()
storage.partial_shutdown_event.set();
storage.replica_is_active_node = nullptr;
LOG_TRACE(log, "Waiting for threads to finish");
LOG_TRACE_FORMATTED(log, "Waiting for threads to finish");
storage.exitLeaderElection();
@ -348,7 +348,7 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown()
storage.cleanup_thread.stop();
storage.part_check_thread.stop();
LOG_TRACE(log, "Threads finished");
LOG_TRACE_FORMATTED(log, "Threads finished");
}
@ -357,7 +357,7 @@ void ReplicatedMergeTreeRestartingThread::shutdown()
/// Stop restarting_thread before stopping other tasks - so that it won't restart them again.
need_stop = true;
task->deactivate();
LOG_TRACE(log, "Restarting thread finished");
LOG_TRACE_FORMATTED(log, "Restarting thread finished");
/// Stop other tasks.
partialShutdown();

View File

@ -573,7 +573,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con
void StorageDistributed::startup()
{
if (remote_database.empty() && !remote_table_function_ptr)
LOG_WARNING(log, "Name of remote database is empty. Default database will be used implicitly.");
LOG_WARNING_FORMATTED(log, "Name of remote database is empty. Default database will be used implicitly.");
if (!volume)
return;

View File

@ -1074,7 +1074,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par
LOG_INFO(log, "Attaching part " << loaded_parts[i]->name << " from " << renamed_parts.old_and_new_names[i].second);
renameTempPartAndAdd(loaded_parts[i], &increment);
renamed_parts.old_and_new_names[i].first.clear();
LOG_INFO(log, "Finished attaching part");
LOG_INFO_FORMATTED(log, "Finished attaching part");
}
/// New parts with other data may appear in place of deleted parts.

View File

@ -221,7 +221,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
{
skip_sanity_checks = true;
LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
LOG_WARNING_FORMATTED(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
}
loadDataParts(skip_sanity_checks);
@ -232,14 +232,14 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
/// Do not activate the replica. It will be readonly.
LOG_ERROR(log, "No ZooKeeper: table will be in readonly mode.");
LOG_ERROR_FORMATTED(log, "No ZooKeeper: table will be in readonly mode.");
is_readonly = true;
return;
}
if (attach && !current_zookeeper->exists(zookeeper_path + "/metadata"))
{
LOG_WARNING(log, "No metadata in ZooKeeper: table will be in readonly mode.");
LOG_WARNING_FORMATTED(log, "No metadata in ZooKeeper: table will be in readonly mode.");
is_readonly = true;
return;
}
@ -579,7 +579,7 @@ void StorageReplicatedMergeTree::createReplica()
if (code == Coordination::Error::ZNODEEXISTS)
throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST);
else if (code == Coordination::Error::ZBADVERSION)
LOG_ERROR(log, "Retrying createReplica(), because some other replicas were created at the same time");
LOG_ERROR_FORMATTED(log, "Retrying createReplica(), because some other replicas were created at the same time");
else
zkutil::KeeperMultiException::check(code, ops, responses);
} while (code == Coordination::Error::ZBADVERSION);
@ -1576,14 +1576,14 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
if (parts_to_add.empty())
{
LOG_INFO(log, "All parts from REPLACE PARTITION command have been already attached");
LOG_INFO_FORMATTED(log, "All parts from REPLACE PARTITION command have been already attached");
tryRemovePartsFromZooKeeperWithRetries(parts_to_remove);
return true;
}
if (parts_to_add.size() < all_parts.size())
{
LOG_WARNING(log, "Some (but not all) parts from REPLACE PARTITION command already exist. REPLACE PARTITION will not be atomic.");
LOG_WARNING_FORMATTED(log, "Some (but not all) parts from REPLACE PARTITION command already exist. REPLACE PARTITION will not be atomic.");
}
StoragePtr source_table;
@ -1839,7 +1839,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo
zkutil::EventPtr event = std::make_shared<Poco::Event>();
if (zookeeper->exists(source_path + "/columns", nullptr, event))
{
LOG_WARNING(log, "Oops, a watch has leaked");
LOG_WARNING_FORMATTED(log, "Oops, a watch has leaked");
break;
}
@ -2451,7 +2451,7 @@ void StorageReplicatedMergeTree::enterLeaderElection()
auto callback = [this]()
{
CurrentMetrics::add(CurrentMetrics::LeaderReplica);
LOG_INFO(log, "Became leader");
LOG_INFO_FORMATTED(log, "Became leader");
is_leader = true;
merge_selecting_task->activateAndSchedule();
@ -2486,7 +2486,7 @@ void StorageReplicatedMergeTree::exitLeaderElection()
if (is_leader)
{
CurrentMetrics::sub(CurrentMetrics::LeaderReplica);
LOG_INFO(log, "Stopped being leader");
LOG_INFO_FORMATTED(log, "Stopped being leader");
is_leader = false;
merge_selecting_task->deactivate();
@ -3277,7 +3277,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer
/// TODO (relax this lock)
auto table_lock = lockExclusively(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations);
LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally.");
LOG_INFO_FORMATTED(log, "Metadata changed in ZooKeeper. Applying changes locally.");
auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this).checkAndFindDiff(metadata_from_entry);
setTableStructure(std::move(columns_from_entry), metadata_diff);
@ -3463,12 +3463,12 @@ void StorageReplicatedMergeTree::alter(
std::vector<String> unwaited;
if (query_context.getSettingsRef().replication_alter_partitions_sync == 2)
{
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
LOG_DEBUG_FORMATTED(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
unwaited = waitForAllReplicasToProcessLogEntry(*alter_entry, false);
}
else if (query_context.getSettingsRef().replication_alter_partitions_sync == 1)
{
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
LOG_DEBUG_FORMATTED(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
waitForReplicaToProcessLogEntry(replica_name, *alter_entry);
}
@ -3477,9 +3477,9 @@ void StorageReplicatedMergeTree::alter(
if (mutation_znode)
{
LOG_DEBUG(log, "Metadata changes applied. Will wait for data changes.");
LOG_DEBUG_FORMATTED(log, "Metadata changes applied. Will wait for data changes.");
waitMutation(*mutation_znode, query_context.getSettingsRef().replication_alter_partitions_sync);
LOG_DEBUG(log, "Data changes applied.");
LOG_DEBUG_FORMATTED(log, "Data changes applied.");
}
}
@ -4519,7 +4519,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const
}
else if (rc == Coordination::ZBADVERSION)
{
LOG_TRACE(log, "Version conflict when trying to create a mutation node, retrying...");
LOG_TRACE_FORMATTED(log, "Version conflict when trying to create a mutation node, retrying...");
continue;
}
else