mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
find {base,src,programs} -name '*.h' -or -name '*.cpp' | xargs grep -l -P 'LOG_\w+\([^,]+, "[^"]+"\)' | xargs sed -i -r -e 's/(LOG_\w+)\(([^,]+, "[^"]+")\)/\1_FORMATTED(\2)/'
This commit is contained in:
parent
3c582fc96f
commit
8d2e80a5e2
@ -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();
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -216,7 +216,7 @@ try
|
||||
attachSystemTables();
|
||||
loadMetadata(*context);
|
||||
DatabaseCatalog::instance().loadDatabases();
|
||||
LOG_DEBUG(log, "Loaded metadata.");
|
||||
LOG_DEBUG_FORMATTED(log, "Loaded metadata.");
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -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))
|
||||
{
|
||||
|
@ -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 (...)
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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.");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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())
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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 (...)
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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!");
|
||||
|
@ -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();
|
||||
|
@ -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");
|
||||
}
|
||||
|
||||
|
||||
|
@ -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));
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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.");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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 {};
|
||||
}
|
||||
|
||||
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user