Merge 'virtual-logger-cnch-dev' into 'cnch-dev'

perf(clickhousech@m-4968955020): VirtualLogger

See merge request: !24002
# Conflicts:
#	programs/udf/manager/Init.h
#	programs/udf/srvstub/UDFImpl.h
#	src/Analyzers/QueryRewriter.cpp
#	src/CloudServices/CnchWorkerServiceImpl.cpp
#	src/CloudServices/CnchWorkerServiceImpl.h
#	src/Common/AdditionalServices.cpp
#	src/Common/CGroup/CGroupManagerFactory.cpp
#	src/Common/Exception.h
#	src/Common/Trace/DirectSystemLogExporter.cpp
#	src/Common/Trace/Telemetry.cpp
#	src/DaemonManager/BGJobStatusInCatalog.h
#	src/DataStreams/ReadDictInputStream.cpp
#	src/DataStreams/ReadDictInputStream.h
#	src/DataStreams/RemoteQueryExecutor.cpp
#	src/DataStreams/RemoteQueryExecutor.h
#	src/DataStreams/UnionBlockInputStream.h
#	src/Disks/DiskByteS3.cpp
#	src/Functions/FunctionFactory.cpp
#	src/Functions/FunctionsBitEngineHelper.h
#	src/Functions/IP2GeoUDF.h
#	src/Functions/UserDefined/UserDefinedExternalFunctionFactory.cpp
#	src/Functions/UserDefined/UserDefinedExternalFunctionFactory.h
#	src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp
#	src/Functions/UserDefined/UserDefinedSQLFunctionFactory.h
#	src/Functions/UserDefined/UserDefinedSQLObjectsLoader.cpp
#	src/Functions/UserDefined/UserDefinedSQLObjectsLoader.h
#	src/IO/CloudFS/CloudFS.cpp
#	src/IO/CloudFS/CloudFS.h
#	src/IO/CloudFS/ReadBufferFromCFS.h
#	src/IO/CloudFS/WriteBufferFromCFS.cpp
#	src/IO/CloudFS/WriteBufferFromCFS.h
#	src/Interpreters/ANNHelper.h
#	src/Interpreters/DistributedStages/PlanSegmentManagerRpcService.h
#	src/Interpreters/InterpreterCreateFunctionQuery.h
#	src/Interpreters/InterpreterDropFunctionQuery.h
#	src/Interpreters/InterpreterSelectQuery.cpp
#	src/Interpreters/TreeRewriter.cpp
#	src/MergeTreeCommon/GlobalGCManager.h
#	src/Optimizer/Rewriter/BitEngineUseLocalDictionary.cpp
#	src/Optimizer/Rewriter/EliminateJoinByForeignKey.cpp
#	src/Storages/BitEngine/BitEngineDictionaryManager.cpp
#	src/Storages/BitEngine/BitEngineDictionaryManager.h
#	src/Storages/BitEngine/BitEngineHelper.cpp
#	src/Storages/BitEngineEncodePartitionHelper.cpp
#	src/Storages/HDFS/HDFSAuth.cpp
#	src/Storages/HDFS/HDFSConfigManager.h
#	src/Storages/Hive/HiveSchemaConverter.h
#	src/Storages/Kafka/KafkaCommon.cpp
#	src/Storages/MemoryDict.h
#	src/Storages/MemoryDictCache.h
#	src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp
#	src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h
#	src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp
#	src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h
#	src/Storages/MergeTree/MergeTreeIndexDiskANN.cpp
#	src/Storages/MergeTree/MergeTreeIndexDiskANN.h
#	src/Storages/MergeTree/MergeTreeIndexFaiss.cpp
#	src/Storages/MergeTree/MergeTreeIndexFaiss.h
#	src/Storages/MergeTree/MergeTreeIndexHNSW.cpp
#	src/Storages/MergeTree/MergeTreeIndexHNSW.h
#	src/Storages/MergeTree/MergeTreeSelectWithSearchProcessor.cpp
#	src/Storages/MergeTree/MergeTreeSelectWithSearchProcessor.h
#	src/Storages/MergeTree/MergeTreeSequentialSource.cpp
#	src/Storages/MergeTree/MergeTreeSequentialSource.h
#	src/Storages/MergeTree/MergeTreeVectorSearchExecutor.cpp
#	src/Storages/MergeTree/MergeTreeVectorSearchExecutor.h
#	src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp
#	src/Storages/MergeTree/S3PartsAttachMeta.h
#	src/Storages/MergeTree/VectorIndex/ANNWhereCondition.cpp
#	src/Storages/MergeTree/VectorIndex/VectorIndexCache.h
#	src/Storages/StorageCnchMergeTree.cpp
This commit is contained in:
fredwang 2024-10-07 03:26:22 +00:00
parent 97cc7fcbef
commit 200f91a6aa
992 changed files with 2429 additions and 1863 deletions

View File

@ -27,6 +27,7 @@
#include <Poco/Logger.h>
#include <Poco/Message.h>
#include <Common/CurrentThread.h>
#include <Common/Logger.h>
namespace
@ -46,16 +47,17 @@ namespace
{ \
const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \
(DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \
if ((logger)->is((PRIORITY)) || _is_clients_log) \
const auto & _logger = (logger); \
if ((_logger)->is((PRIORITY)) || _is_clients_log) \
{ \
std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \
if (auto _channel = (logger)->getChannel()) \
if (auto _channel = (_logger)->getChannel()) \
{ \
std::string file_function; \
file_function += __FILE__; \
file_function += "; "; \
file_function += __PRETTY_FUNCTION__; \
Poco::Message poco_message((logger)->name(), formatted_message, \
Poco::Message poco_message((_logger)->name(), formatted_message, \
(PRIORITY), file_function.c_str(), __LINE__); \
_channel->log(poco_message); \
} \

View File

@ -87,9 +87,9 @@ static void InitMetricsHelper(const metrics2::MetricCollectorConf& config, const
metrics2::Metrics::init(config);
isInitialized = true;
LOG_INFO(&Poco::Logger::get("metric_helper::InitMetricsHelper"), "{}, tag = {}", config.toString(), tags);
LOG_INFO(getLogger("metric_helper::InitMetricsHelper"), "{}, tag = {}", config.toString(), tags);
// LOG_INFO(&Poco::Logger::get("metric_helper::InitMetricsHelper"), config.toString() << ", tag = " << tags);
// LOG_INFO(getLogger("metric_helper::InitMetricsHelper"), config.toString() << ", tag = " << tags);
}
void InitMetrics(const metrics2::MetricCollectorConf& config, const std::string & custom_tags) {

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include "Aliases.h"
#include "Internals.h"
#include "TaskCluster.h"
@ -19,7 +20,7 @@ public:
const String & host_id_,
const String & proxy_database_name_,
ContextMutablePtr context_,
Poco::Logger * log_)
LoggerPtr log_)
: WithMutableContext(context_),
task_zookeeper_path(task_path_),
host_id(host_id_),
@ -216,7 +217,7 @@ private:
bool experimental_use_sample_offset{false};
Poco::Logger * log;
LoggerPtr log;
std::chrono::milliseconds default_sleep_time{1000};
};

View File

@ -130,7 +130,7 @@ void ClusterCopierApp::mainImpl()
StatusFile status_file(process_path + "/status", StatusFile::write_full_info);
ThreadStatus thread_status;
auto * log = &logger();
auto log = getLogger(logger());
LOG_INFO(log, "Starting clickhouse-copier (id {}, host_id {}, path {}, revision {})", process_id, host_id, process_path, ClickHouseRevision::getVersionRevision());
SharedContextHolder shared_context = Context::createShared();

View File

@ -1,5 +1,7 @@
#pragma once
#include <Common/Logger.h>
/** Allows to compare two incremental counters of type UInt32 in presence of possible overflow.
* We assume that we compare values that are not too far away.
* For example, when we increment 0xFFFFFFFF, we get 0. So, 0xFFFFFFFF is less than 0.
@ -177,7 +179,7 @@ public:
auto watch_callback =
[stale = stale] (const Coordination::WatchResponse & rsp)
{
auto logger = &Poco::Logger::get("ClusterCopier");
auto logger = getLogger("ClusterCopier");
if (rsp.error == Coordination::Error::ZOK)
{
switch (rsp.type)

View File

@ -1,4 +1,5 @@
// #include <Storages/MergeTree/MergeTreeDataPart.h>
#include <Common/Logger.h>
#include <Storages/StorageCloudMergeTree.h>
#include <Poco/Logger.h>
#include <Common/Exception.h>
@ -101,7 +102,7 @@ public:
void removeDumpVersionFromZk(const Context & context);
void setLog(Poco::Logger * log_) { log = log_; }
void setLog(LoggerPtr log_) { log = log_; }
private:
void writeTempUniqueKeyIndex(Block & block, size_t first_rid, rocksdb::DB & temp_index, StorageCloudMergeTree & cloud);
@ -111,7 +112,7 @@ private:
String unique_version_column;
String dump_lsn_path;
ManifestStore manifest_store;
Poco::Logger * log;
LoggerPtr log;
};
}

View File

@ -135,7 +135,7 @@ private:
ContextMutablePtr global_context;
Settings settings;
Int64 current_shard_number {0};
Poco::Logger * log{};
LoggerPtr log{};
UniqueTableDumpHelper unique_table_dump_helper;
};
@ -812,8 +812,8 @@ int ClickHouseDumper::main(const std::vector<String> &)
config().add(config_processor.loadConfig().configuration.duplicate(), PRIO_APPLICATION, true, false);
}
log = &logger();
log->setLevel(config().getString("logger.level", "debug"));
logger().setLevel(config().getString("logger.level", "debug"));
log = getLogger(logger());
unique_table_dump_helper.setLog(log);
shared_context = DB::Context::createShared();

View File

@ -48,8 +48,9 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
po::store(po::command_line_parser(argc, argv).options(desc).run(), options);
Poco::AutoPtr<Poco::ConsoleChannel> console_channel(new Poco::ConsoleChannel);
Poco::Logger * logger = &Poco::Logger::get("KeeperConverter");
logger->setChannel(console_channel);
LoggerRawPtr raw_logger = getRawLogger("KeeperConverter");
raw_logger->setChannel(console_channel);
LoggerPtr logger = getLogger(*raw_logger);
if (options.count("help"))
{

View File

@ -122,7 +122,7 @@ int waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t
return current_connections;
}
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log)
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, LoggerPtr log)
{
Poco::Net::SocketAddress socket_address;
try
@ -186,7 +186,7 @@ std::string getUserName(uid_t user_id)
Poco::Net::SocketAddress Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const
{
auto address = makeSocketAddress(host, port, &logger());
auto address = makeSocketAddress(host, port, getLogger(logger()));
#if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100
if (secure)
/// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl
@ -313,7 +313,7 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options)
int Keeper::main(const std::vector<std::string> & /*args*/)
{
Poco::Logger * log = &logger();
LoggerPtr log = getLogger(logger());
UseSSL use_ssl;

View File

@ -21,6 +21,7 @@
#pragma once
#include <Common/Logger.h>
#include <Server/IServer.h>
#include <daemon/BaseDaemon.h>

View File

@ -13,7 +13,7 @@ CatBoostLibraryHandlerFactory & CatBoostLibraryHandlerFactory::instance()
}
CatBoostLibraryHandlerFactory::CatBoostLibraryHandlerFactory()
: log(&Poco::Logger::get("CatBoostLibraryHandlerFactory"))
: log(getLogger("CatBoostLibraryHandlerFactory"))
{
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include "CatBoostLibraryHandler.h"
#include <common/defines.h>
@ -31,7 +32,7 @@ private:
/// map: model path --> catboost library handler
std::unordered_map<String, CatBoostLibraryHandlerPtr> library_handlers TSA_GUARDED_BY(mutex);
std::mutex mutex;
Poco::Logger * log;
LoggerPtr log;
};
}

View File

@ -26,7 +26,7 @@ void ExternalDictionaryLibraryHandlerFactory::create(
if (library_handlers.contains(dictionary_id))
{
LOG_WARNING(&Poco::Logger::get("ExternalDictionaryLibraryHandlerFactory"), "Library handler with dictionary id {} already exists", dictionary_id);
LOG_WARNING(getLogger("ExternalDictionaryLibraryHandlerFactory"), "Library handler with dictionary id {} already exists", dictionary_id);
return;
}

View File

@ -12,7 +12,7 @@ LibraryBridgeHandlerFactory::LibraryBridgeHandlerFactory(
size_t keep_alive_timeout_,
ContextPtr context_)
: WithContext(context_)
, log(&Poco::Logger::get(name_))
, log(getLogger(name_))
, name(name_)
, keep_alive_timeout(keep_alive_timeout_)
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <common/logger_useful.h>
@ -19,7 +20,7 @@ public:
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
private:
Poco::Logger * log;
LoggerPtr log;
const std::string name;
const size_t keep_alive_timeout;
};

View File

@ -46,7 +46,7 @@ namespace
if (!response.sent())
*response.send() << message << std::endl;
LOG_WARNING(&Poco::Logger::get("LibraryBridge"), (message));
LOG_WARNING(getLogger("LibraryBridge"), (message));
}
std::shared_ptr<Block> parseColumns(String && column_string)
@ -93,7 +93,7 @@ static void writeData(Block data, OutputFormatPtr format)
ExternalDictionaryLibraryBridgeRequestHandler::ExternalDictionaryLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, keep_alive_timeout(keep_alive_timeout_)
, log(&Poco::Logger::get("ExternalDictionaryLibraryBridgeRequestHandler"))
, log(getLogger("ExternalDictionaryLibraryBridgeRequestHandler"))
{
}
@ -385,7 +385,7 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ
ExternalDictionaryLibraryBridgeExistsHandler::ExternalDictionaryLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, keep_alive_timeout(keep_alive_timeout_)
, log(&Poco::Logger::get("ExternalDictionaryLibraryBridgeExistsHandler"))
, log(getLogger("ExternalDictionaryLibraryBridgeExistsHandler"))
{
}
@ -424,7 +424,7 @@ CatBoostLibraryBridgeRequestHandler::CatBoostLibraryBridgeRequestHandler(
size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, keep_alive_timeout(keep_alive_timeout_)
, log(&Poco::Logger::get("CatBoostLibraryBridgeRequestHandler"))
, log(getLogger("CatBoostLibraryBridgeRequestHandler"))
{
}
@ -622,7 +622,7 @@ void CatBoostLibraryBridgeRequestHandler::handleRequest(HTTPServerRequest & requ
CatBoostLibraryBridgeExistsHandler::CatBoostLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, keep_alive_timeout(keep_alive_timeout_)
, log(&Poco::Logger::get("CatBoostLibraryBridgeExistsHandler"))
, log(getLogger("CatBoostLibraryBridgeExistsHandler"))
{
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <common/logger_useful.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandler.h>
@ -26,7 +27,7 @@ private:
static constexpr inline auto FORMAT = "RowBinary";
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
@ -40,7 +41,7 @@ public:
private:
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
@ -69,7 +70,7 @@ public:
private:
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
@ -83,7 +84,7 @@ public:
private:
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
}

View File

@ -153,7 +153,7 @@ void LocalServer::tryInitPath()
{
// The path is not provided explicitly - use a unique path in the system temporary directory
// (or in the current dir if temporary don't exist)
Poco::Logger * log = &logger();
LoggerPtr log = getLogger("LocalServer");
std::filesystem::path parent_folder;
std::filesystem::path default_path;
@ -217,7 +217,7 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str
int LocalServer::main(const std::vector<std::string> & /*args*/)
try
{
Poco::Logger * log = &logger();
LoggerPtr log = getLogger("LocalServer");
ThreadStatus thread_status;
UseSSL use_ssl;

View File

@ -2,6 +2,7 @@
#if USE_ODBC
#include <Common/Logger.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandler.h>
@ -17,7 +18,7 @@ class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext
public:
ODBCColumnsInfoHandler(size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, log(&Poco::Logger::get("ODBCColumnsInfoHandler"))
, log(getLogger("ODBCColumnsInfoHandler"))
, keep_alive_timeout(keep_alive_timeout_)
{
}
@ -25,7 +26,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
private:
Poco::Logger * log;
LoggerPtr log;
size_t keep_alive_timeout;
};

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context_fwd.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include "ColumnInfoHandler.h"
@ -19,7 +20,7 @@ class ODBCBridgeHandlerFactory : public HTTPRequestHandlerFactory, WithContext
public:
ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, log(&Poco::Logger::get(name_))
, log(getLogger(name_))
, name(name_)
, keep_alive_timeout(keep_alive_timeout_)
{
@ -28,7 +29,7 @@ public:
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
private:
Poco::Logger * log;
LoggerPtr log;
std::string name;
size_t keep_alive_timeout;
};

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandler.h>
@ -16,7 +17,7 @@ class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext
public:
IdentifierQuoteHandler(size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, log(&Poco::Logger::get("IdentifierQuoteHandler"))
, log(getLogger("IdentifierQuoteHandler"))
, keep_alive_timeout(keep_alive_timeout_)
{
}
@ -24,7 +25,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
private:
Poco::Logger * log;
LoggerPtr log;
size_t keep_alive_timeout;
};

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context_fwd.h>
#include <Server/HTTP/HTTPRequestHandler.h>
#include <Poco/Logger.h>
@ -24,7 +25,7 @@ public:
ContextPtr context_,
const String & mode_)
: WithContext(context_)
, log(&Poco::Logger::get("ODBCHandler"))
, log(getLogger("ODBCHandler"))
, keep_alive_timeout(keep_alive_timeout_)
, mode(mode_)
{
@ -33,7 +34,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
private:
Poco::Logger * log;
LoggerPtr log;
size_t keep_alive_timeout;
String mode;

View File

@ -21,7 +21,7 @@ namespace ErrorCodes
ODBCBlockInputStream::ODBCBlockInputStream(
nanodbc::ConnectionHolderPtr connection_holder, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_)
: log(&Poco::Logger::get("ODBCBlockInputStream"))
: log(getLogger("ODBCBlockInputStream"))
, max_block_size{max_block_size_}
, query(query_str)
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <string>
#include <Core/Block.h>
#include <DataStreams/IBlockInputStream.h>
@ -32,7 +33,7 @@ private:
column.insertFrom(sample_column, 0);
}
Poco::Logger * log;
LoggerPtr log;
const UInt64 max_block_size;
ExternalResultDescription description;

View File

@ -46,7 +46,7 @@ ODBCBlockOutputStream::ODBCBlockOutputStream(nanodbc::ConnectionHolderPtr connec
const Block & sample_block_,
ContextPtr local_context_,
IdentifierQuotingStyle quoting_)
: log(&Poco::Logger::get("ODBCBlockOutputStream"))
: log(getLogger("ODBCBlockOutputStream"))
, connection_holder(std::move(connection_holder_))
, db_name(remote_database_name_)
, table_name(remote_table_name_)

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Core/Block.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Core/ExternalResultDescription.h>
@ -27,7 +28,7 @@ public:
void write(const Block & block) override;
private:
Poco::Logger * log;
LoggerPtr log;
nanodbc::ConnectionHolderPtr connection_holder;
std::string db_name;

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandler.h>
#include <Poco/Logger.h>
@ -18,7 +19,7 @@ class SchemaAllowedHandler : public HTTPRequestHandler, WithContext
public:
SchemaAllowedHandler(size_t keep_alive_timeout_, ContextPtr context_)
: WithContext(context_)
, log(&Poco::Logger::get("SchemaAllowedHandler"))
, log(getLogger("SchemaAllowedHandler"))
, keep_alive_timeout(keep_alive_timeout_)
{
}
@ -26,7 +27,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
private:
Poco::Logger * log;
LoggerPtr log;
size_t keep_alive_timeout;
};

View File

@ -26,7 +26,7 @@ std::string getIdentifierQuote(nanodbc::ConnectionHolderPtr connection_holder)
}
catch (...)
{
LOG_WARNING(&Poco::Logger::get("ODBCGetIdentifierQuote"), "Cannot fetch identifier quote. Default double quote is used. Reason: {}", getCurrentExceptionMessage(false));
LOG_WARNING(getLogger("ODBCGetIdentifierQuote"), "Cannot fetch identifier quote. Default double quote is used. Reason: {}", getCurrentExceptionMessage(false));
return "\"";
}

View File

@ -45,7 +45,7 @@ const std::string PartMergerApp::default_config = "<yandex>\n"
"<merge_selector>merger</merge_selector>\n"
"</yandex>";
void PartMergerApp::initHDFS(DB::ContextMutablePtr context, Poco::Logger * log)
void PartMergerApp::initHDFS(DB::ContextMutablePtr context, LoggerPtr log)
{
LOG_DEBUG(log, "Initialize HDFS driver.");
using HDFSConnectionParams = DB::HDFSConnectionParams;
@ -109,7 +109,7 @@ int PartMergerApp::main([[maybe_unused]] const std::vector<DB::String> & args)
{
Poco::Logger::root().setLevel("information");
}
auto * log = &Poco::Logger::get("PartMergerApp");
auto log = getLogger("PartMergerApp");
LOG_DEBUG(log, "Parse arguments");
// Parse arguments.

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <sstream>
#include <string>
#include <Poco/FileChannel.h>
@ -54,12 +55,12 @@ private:
/**
* Print help message for part_merger_tool.
*/
inline static void mergerHelp(Poco::Logger * log) { LOG_ERROR(log, PartMergerApp::help_message); }
inline static void mergerHelp(LoggerPtr log) { LOG_ERROR(log, PartMergerApp::help_message); }
/**
* Init HDFS default configuration.
*/
void initHDFS(DB::ContextMutablePtr context, Poco::Logger * log);
void initHDFS(DB::ContextMutablePtr context, LoggerPtr log);
int main([[maybe_unused]] const std::vector<DB::String> & args) override;

View File

@ -52,7 +52,7 @@ int mainHelp(int, char **)
return 0;
}
void run(const std::string & query, Poco::Logger * log)
void run(const std::string & query, LoggerPtr log)
{
LOG_DEBUG(log, "Executing query : {}", query);
DB::ThreadStatus status;
@ -133,7 +133,7 @@ int mainEntryClickhousePartToolkit(int argc, char ** argv)
{
}
Poco::Logger * log = &Poco::Logger::get("part-toolkit");
LoggerPtr log = getLogger("part-toolkit");
LOG_INFO(log, "Logger level: {}", log_level);

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <CloudServices/CnchServerServiceImpl.h>
#include <CloudServices/CnchWorkerServiceImpl.h>
#include <Interpreters/Context.h>
@ -45,7 +46,7 @@ public:
else if (global_context->getServerType() == ServerType::cnch_worker)
{
addService(*rpc_server, rpc_services, CnchWorkerServiceImpl_RegisterService(global_context).service);
LOG_DEBUG(&Poco::Logger::get("BrpcServerHolder"), "Start register RemoteDiskCacheService: {}", host_port);
LOG_DEBUG(getLogger("BrpcServerHolder"), "Start register RemoteDiskCacheService: {}", host_port);
addService(*rpc_server, rpc_services, RemoteDiskCacheService_RegisterService(global_context).service);
}
@ -61,7 +62,7 @@ public:
{
start_success = false;
if (listen_try)
LOG_ERROR(&Poco::Logger::get("BrpcServerHolder"), "Failed tp start rpc server on {}", host_port);
LOG_ERROR(getLogger("BrpcServerHolder"), "Failed tp start rpc server on {}", host_port);
else
throw Exception("Failed tp start rpc server on " + host_port, ErrorCodes::BRPC_EXCEPTION);
}

View File

@ -325,7 +325,7 @@ static std::string getUserName(uid_t user_id)
return toString(user_id);
}
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log)
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, LoggerPtr log)
{
Poco::Net::SocketAddress socket_address;
try
@ -355,7 +355,7 @@ Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port
Poco::Net::SocketAddress Server::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const
{
auto address = makeSocketAddress(host, port, &logger());
auto address = makeSocketAddress(host, port, getLogger(logger()));
#if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100
if (secure)
/// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl
@ -403,7 +403,7 @@ static void clearOldStoreDirectory(const DisksMap& disk_map)
try
{
LOG_DEBUG(&Poco::Logger::get(__func__), "Removing {} from disk {}",
LOG_DEBUG(getLogger(__func__), "Removing {} from disk {}",
String(fs::path(disk->getPath()) / iter->path()), disk->getName());
disk->removeRecursive(iter->path());
}
@ -508,7 +508,7 @@ void checkForUsersNotInMainConfig(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_path,
const std::string & users_config_path,
Poco::Logger * log)
LoggerPtr log)
{
if (config.getBool("skip_check_for_incorrect_settings", false))
return;
@ -539,7 +539,7 @@ void checkForUsersNotInMainConfig(
void huallocLogPrint(std::string s)
{
static Poco::Logger * logger = &Poco::Logger::get("HuallocDebug");
static LoggerPtr logger = getLogger("HuallocDebug");
LOG_INFO(logger, s);
}
@ -571,7 +571,7 @@ void limitMemoryCacheDefaultMaxRatio(RootConfiguration & root_config, const UInt
Float32 max_total_ratio = root_config.cache_size_to_ram_max_ratio.value;
Float32 lowered_ratio = (total_ratio > max_total_ratio ? max_total_ratio / total_ratio : 1.0f);
Poco::Logger * logger = &Poco::Logger::get("MemoryCacheDefaultRatioLimit");
auto logger = getLogger("MemoryCacheDefaultRatioLimit");
LOG_INFO(logger, "Total memory {}, max ratio for memory cache is {}{}",
formatReadableSizeWithBinarySuffix(memory_amount), max_total_ratio,
@ -601,7 +601,7 @@ void limitMemoryCacheDefaultMaxRatio(RootConfiguration & root_config, const UInt
int Server::main(const std::vector<std::string> & /*args*/)
{
Poco::Logger * log = &logger();
LoggerPtr log = getLogger(logger());
UseSSL use_ssl;

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Server/IServer.h>
#include <daemon/BaseDaemon.h>

View File

@ -344,7 +344,7 @@ void ParallelInspectRunner::InspectTask::exec() {
}
ParallelInspectRunner::TaskAllocator::TaskAllocator(const String& base_path, FSOp& fs_op, InspectTask::Type type,
const String& stream_name, Poco::Logger* logger):
const String& stream_name, LoggerPtr logger):
base_path_(base_path), fs_op_(fs_op), type_(type), logger_(logger), stream_name_(stream_name) {
if (fs_op_.isFile(base_path_)) {
abs_paths_.push_back(base_path_);
@ -382,7 +382,7 @@ void ParallelInspectRunner::TaskAllocator::collectPaths(const String& base_path,
ParallelInspectRunner::ParallelInspectRunner(const String& base_path, FSOp& fs_op,
size_t worker_threads, InspectTask::Type type, const String& stream_name):
task_allocator_(base_path, fs_op, type, stream_name, &Poco::Logger::get("ParallelInspectRunner")) {
task_allocator_(base_path, fs_op, type, stream_name, getLogger("ParallelInspectRunner")) {
worker_pool_ = std::make_unique<ThreadPool>(worker_threads, worker_threads, worker_threads);
for (size_t i = 0; i < worker_threads; ++i) {

View File

@ -1,3 +1,4 @@
#include <Common/Logger.h>
#include <iostream>
#include <mutex>
#include <memory>
@ -129,14 +130,14 @@ public:
ALL = BRIEF | CHECKSUMS,
};
InspectTask(const String& path, FSOp& fs, Type type, const String& stream_name, Poco::Logger* logger):
InspectTask(const String& path, FSOp& fs, Type type, const String& stream_name, LoggerPtr logger):
type_(type), logger_(logger), stream_name_(stream_name), inspector_(path, fs) {}
void exec();
private:
Type type_;
Poco::Logger* logger_;
LoggerPtr logger_;
String stream_name_;
PartInspector inspector_;
};
@ -144,7 +145,7 @@ public:
class TaskAllocator {
public:
TaskAllocator(const String& base_path, FSOp& fs_op, InspectTask::Type type,
const String& stream_name, Poco::Logger* logger);
const String& stream_name, LoggerPtr logger);
std::unique_ptr<InspectTask> acquire();
@ -157,7 +158,7 @@ public:
InspectTask::Type type_;
Poco::Logger* logger_;
LoggerPtr logger_;
String stream_name_;

View File

@ -483,7 +483,7 @@ public:
optimizeTree();
}
void logTree(Poco::Logger * log, const String & title) const
void logTree(LoggerPtr log, const String & title) const
{
LOG_TRACE(log, "Tree({}): level={}, name={}, flags={}, min_flags={}, max_flags={}, num_children={}, is_sensitive={}",
title, level, node_name ? *node_name : "NULL", flags.toString(),
@ -1250,7 +1250,7 @@ AccessRights AccessRights::getFullAccess()
template <bool IsSensitive>
void AccessRightsBase<IsSensitive>::logTree() const
{
auto * log = &Poco::Logger::get("AccessRights");
auto log = getLogger("AccessRights");
if (root)
{
root->logTree(log, "");

View File

@ -247,7 +247,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
throw;
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
&Poco::Logger::get("AddressPatterns"),
getLogger("AddressPatterns"),
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
client_address.toString(), e.displayText(), e.code());
return false;
@ -280,7 +280,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
throw;
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
&Poco::Logger::get("AddressPatterns"),
getLogger("AddressPatterns"),
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
client_address.toString(), e.displayText(), e.code());
return false;

View File

@ -357,7 +357,7 @@ void ContextAccess::setUser(const UserPtr & user_) const
}
user_name = user->getName();
trace_log = &Poco::Logger::get("ContextAccess (" + user_name + ")");
trace_log = getLogger("ContextAccess (" + user_name + ")");
std::vector<UUID> current_roles, current_roles_with_admin_option;

View File

@ -2,11 +2,12 @@
#include <Access/AccessRights.h>
#include <Access/RowPolicy.h>
#include <Interpreters/ClientInfo.h>
#include <Common/Logger.h>
#include <Core/UUID.h>
#include <Interpreters/ClientInfo.h>
#include <boost/container/flat_set.hpp>
#include <common/scope_guard.h>
#include <common/shared_ptr_helper.h>
#include <boost/container/flat_set.hpp>
#include <mutex>
@ -223,7 +224,7 @@ private:
const AccessControlManager * manager = nullptr;
const Params params;
bool is_full_access = false;
mutable Poco::Logger * trace_log = nullptr;
mutable LoggerPtr trace_log = nullptr;
mutable UserPtr user;
mutable String user_name;
mutable scope_guard subscription_for_user_change;

View File

@ -199,7 +199,7 @@ namespace
}
AccessEntityPtr tryReadEntityFile(const String & file_path, Poco::Logger & log)
AccessEntityPtr tryReadEntityFile(const String & file_path, LoggerPtr log)
{
try
{
@ -207,7 +207,7 @@ namespace
}
catch (...)
{
tryLogCurrentException(&log, "Could not parse " + file_path);
tryLogCurrentException(log, "Could not parse " + file_path);
return nullptr;
}
}
@ -560,7 +560,7 @@ bool DiskAccessStorage::rebuildLists()
continue;
const auto access_entity_file_path = getEntityFilePath(directory_path, id);
auto entity = tryReadEntityFile(access_entity_file_path, *getLogger());
auto entity = tryReadEntityFile(access_entity_file_path, getLogger());
if (!entity)
continue;

View File

@ -239,7 +239,7 @@ void ExternalAuthenticators::reset()
kerberos_params.reset();
}
void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log)
void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfiguration & config, LoggerPtr log)
{
std::scoped_lock lock(mutex);
reset();

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Access/LDAPClient.h>
#include <Access/Credentials.h>
#include <Access/GSSAcceptor.h>
@ -30,7 +31,7 @@ class ExternalAuthenticators
{
public:
void reset();
void setConfiguration(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log);
void setConfiguration(const Poco::Util::AbstractConfiguration & config, LoggerPtr log);
// The name and readiness of the credentials must be verified before calling these.
bool checkLDAPCredentials(const String & server, const BasicCredentials & credentials,

View File

@ -337,7 +337,7 @@ void GSSAcceptorContext::initHandles()
}
}
String GSSAcceptorContext::processToken(const String & input_token, Poco::Logger * log)
String GSSAcceptorContext::processToken(const String & input_token, LoggerPtr log)
{
std::scoped_lock lock(gss_global_mutex);
@ -459,7 +459,7 @@ void GSSAcceptorContext::initHandles()
{
}
String GSSAcceptorContext::processToken(const String &, Poco::Logger *)
String GSSAcceptorContext::processToken(const String &, LoggerPtr)
{
throw Exception("ClickHouse was built without GSS-API/Kerberos support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME);
}

View File

@ -5,6 +5,7 @@
#endif
#include <Access/Credentials.h>
#include <Common/Logger.h>
#include <common/types.h>
#include <memory>
@ -42,7 +43,7 @@ public:
const String & getRealm() const;
bool isFailed() const;
MAYBE_NORETURN String processToken(const String & input_token, Poco::Logger * log);
MAYBE_NORETURN String processToken(const String & input_token, LoggerPtr log);
private:
void reset();

View File

@ -538,12 +538,12 @@ UUID IAccessStorage::generateRandomID()
}
Poco::Logger * IAccessStorage::getLogger() const
LoggerPtr IAccessStorage::getLogger() const
{
Poco::Logger * ptr = log.load();
if (!ptr)
log.store(ptr = &Poco::Logger::get("Access(" + storage_name + ")"), std::memory_order_relaxed);
return ptr;
callOnce(log_initialized, [&] {
log = ::getLogger("Access(" + storage_name + ")");
});
return log;
}

View File

@ -1,6 +1,8 @@
#pragma once
#include <Access/IAccessEntity.h>
#include <Common/Logger.h>
#include <Common/callOnce.h>
#include <Core/Types.h>
#include <Core/UUID.h>
#include <common/scope_guard.h>
@ -10,7 +12,6 @@
#include <atomic>
namespace Poco { class Logger; }
namespace Poco::Net { class IPAddress; }
namespace DB
@ -178,7 +179,7 @@ protected:
virtual UUID getIDOfLoggedUserImpl(const String & user_name) const;
static UUID generateRandomID();
Poco::Logger * getLogger() const;
LoggerPtr getLogger() const;
static String outputEntityTypeAndName(EntityType type, const String & name) { return EntityTypeInfo::get(type).outputWithEntityName(name); }
[[noreturn]] void throwNotFound(const UUID & id) const;
[[noreturn]] void throwNotFound(EntityType type, const String & name) const;
@ -200,7 +201,8 @@ protected:
private:
const String storage_name;
mutable std::atomic<Poco::Logger *> log = nullptr;
mutable OnceFlag log_initialized;
mutable LoggerPtr log = nullptr;
};

View File

@ -87,7 +87,7 @@ String KerberosInit::fmtError(krb5_error_code code) const
void KerberosInit::init(const String & keytab_file, const String & principal, const String & cache_name)
{
auto * log = &Poco::Logger::get("KerberosInit");
auto log = getLogger("KerberosInit");
LOG_TRACE(log,"Trying to authenticate with Kerberos v5");
krb5_error_code ret;

View File

@ -475,7 +475,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params)
for (std::size_t i = 0; referrals[i]; i++)
{
LOG_WARNING(&Poco::Logger::get("LDAPClient"), "Received reference during LDAP search but not following it: {}", referrals[i]);
LOG_WARNING(getLogger("LDAPClient"), "Received reference during LDAP search but not following it: {}", referrals[i]);
}
}

View File

@ -105,7 +105,7 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_)
catch (...)
{
tryLogCurrentException(
&Poco::Logger::get("RowPolicy"),
getLogger("RowPolicy"),
String("Could not parse the condition ") + toString(type) + " of row policy "
+ backQuote(policy->getName()));
}

View File

@ -45,7 +45,7 @@ namespace SaslCommon
static int saslLogCallbacks(void * context, int level, const char * message)
{
String auth_context = reinterpret_cast<char *>(context);
auto * log = &Poco::Logger::get(auth_context);
auto log = getLogger(auth_context);
const String auth_message(message);
if (!message)
@ -179,7 +179,7 @@ void SaslClient::saslInit(sasl_callback_t * callbacks)
{
if (SaslCommon::sasl_inited)
{
LOG_WARNING(&Poco::Logger::get("SaslClient"), "Sasl Client is already Init");
LOG_WARNING(getLogger("SaslClient"), "Sasl Client is already Init");
return;
}
int result = sasl_client_init(callbacks);

View File

@ -2,6 +2,7 @@
#include <Advisor/Rules/WorkloadAdvisor.h>
#include <Advisor/WorkloadTable.h>
#include <Common/Logger.h>
#include <Core/Types.h>
#include <Interpreters/Context_fwd.h>
#include <Parsers/ASTAdviseQuery.h>
@ -24,7 +25,7 @@ public:
static WorkloadAdvisors getAdvisors(ASTAdviseQuery::AdvisorType type);
ASTAdviseQuery::AdvisorType type;
Poco::Logger * log = &Poco::Logger::get("Advisor");
LoggerPtr log = getLogger("Advisor");
};
}

View File

@ -3,6 +3,7 @@
#include <Advisor/AdvisorContext.h>
#include <Advisor/Rules/WorkloadAdvisor.h>
#include <Analyzers/QualifiedColumnName.h>
#include <Common/Logger.h>
#include <Core/Types.h>
#include <Poco/Logger.h>
@ -17,7 +18,7 @@ public:
private:
bool isValidColumn(const QualifiedColumnName & column, AdvisorContext & context) const;
Poco::Logger * log = &Poco::Logger::get("ClusterKeyAdvisor");
LoggerPtr log = getLogger("ClusterKeyAdvisor");
};
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Advisor/AdvisorContext.h>
#include <Advisor/Rules/WorkloadAdvisor.h>
#include <Advisor/SignatureUsage.h>
@ -53,7 +54,7 @@ private:
const OutputType output_type;
const bool only_aggregate;
const bool ignore_filter;
Poco::Logger * log = &Poco::Logger::get("MaterializedViewAdvisor");
LoggerPtr log = getLogger("MaterializedViewAdvisor");
};
/**

View File

@ -185,7 +185,7 @@ private:
double optimal_cost;
std::unordered_set<TableLayout, TableLayoutHash> explored;
Poco::Logger * log = &Poco::Logger::get("PartitionKeyAdvisor");
LoggerPtr log = getLogger("PartitionKeyAdvisor");
};

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Advisor/Rules/WorkloadAdvisor.h>
#include <Advisor/AdvisorContext.h>
#include <Advisor/WorkloadTable.h>
@ -27,7 +28,7 @@ private:
std::vector<QualifiedColumnName> getSortedInterestingColumns(AdvisorContext & context) const;
bool isValidColumn(const QualifiedColumnName & column, AdvisorContext & context) const;
Poco::Logger * log = &Poco::Logger::get("PartitionKeyAdvisor");
LoggerPtr log = getLogger("PartitionKeyAdvisor");
static constexpr bool enable_memo_based_advise = 1;
};

View File

@ -134,7 +134,7 @@ WorkloadQueries WorkloadQuery::build(const std::vector<std::string> & queries, c
setThreadName("BuildQuery");
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
LOG_DEBUG(&Poco::Logger::get("WorkloadQuery"), "start building query {}", i);
LOG_DEBUG(getLogger("WorkloadQuery"), "start building query {}", i);
const auto & query = queries[i];
try
{
@ -142,7 +142,7 @@ WorkloadQueries WorkloadQuery::build(const std::vector<std::string> & queries, c
res[i] = std::move(workload_query);
} catch (Exception & e)
{
LOG_WARNING(&Poco::Logger::get("WorkloadQuery"),
LOG_WARNING(getLogger("WorkloadQuery"),
"failed to build query, reason: {}, sql: {}",
e.message(), query);
}
@ -150,7 +150,7 @@ WorkloadQueries WorkloadQuery::build(const std::vector<std::string> & queries, c
}
query_thread_pool.wait();
res.erase(std::remove(res.begin(), res.end(), nullptr), res.end());
LOG_DEBUG(&Poco::Logger::get("WorkloadQuery"), "built queries {}/{}", res.size(), queries.size());
LOG_DEBUG(getLogger("WorkloadQuery"), "built queries {}/{}", res.size(), queries.size());
return res;
}

View File

@ -42,7 +42,7 @@ WorkloadTableStats WorkloadTableStats::build(ContextPtr context, const String &
collector.readAllFromCatalog();
basic_stats = collector.toPlanNodeStatistics().value_or(nullptr);
if (basic_stats)
LOG_DEBUG(&Poco::Logger::get("WorkloadTableStats"), "Stats for table {}.{}: {} rows, {} symbols",
LOG_DEBUG(getLogger("WorkloadTableStats"), "Stats for table {}.{}: {} rows, {} symbols",
database_name, table_name, basic_stats->getRowCount(), basic_stats->getSymbolStatistics().size());
} catch (...) {}
@ -80,7 +80,7 @@ WorkloadExtendedStatsPtr WorkloadTableStats::collectExtendedStats(
query.pop_back();
query += fmt::format(" FROM {}.{}", database, table);
LOG_DEBUG(&Poco::Logger::get("WorkloadTableStats"), "Collecting extended stats for table: {}", query);
LOG_DEBUG(getLogger("WorkloadTableStats"), "Collecting extended stats for table: {}", query);
Statistics::SubqueryHelper subquery_helper = Statistics::SubqueryHelper::create(context, query);
Block result = Statistics::getOnlyRowFrom(subquery_helper);

View File

@ -15,6 +15,7 @@
#pragma once
#include <Common/Logger.h>
#include <pdqsort.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnArray.h>
@ -124,7 +125,7 @@ struct AggregateFunctionPathSplitData
s += "Event(index=" + std::to_string(events[i].index) + ", time=" + std::to_string(events[i].time)
+ ", param=" + events[i].param.toString() + ").\n";
}
LOG_DEBUG(&Poco::Logger::get("AggregateFunctionPathSplit"), "events:" + s + ".");
LOG_DEBUG(getLogger("AggregateFunctionPathSplit"), "events:" + s + ".");
}
};

View File

@ -15,6 +15,7 @@
#pragma once
#include <Common/Logger.h>
#include <pdqsort.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnArray.h>
@ -125,7 +126,7 @@ struct AggregateFunctionSessionAnalysisData
String s = "Event size: " + std::to_string(events.size()) + "\n";
for (const auto & event : events)
s += "Event(type=" + std::to_string(event.type) + ", time=" + std::to_string(event.time) + ", value=" + event.value.toString() + ")\n";
LOG_DEBUG(&Poco::Logger::get("AggregateFunctionSessionAnalysis"), "events:" + s + ".");
LOG_DEBUG(getLogger("AggregateFunctionSessionAnalysis"), "events:" + s + ".");
}
};

View File

@ -15,6 +15,7 @@
#pragma once
#include <Common/Logger.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Common/Stopwatch.h>
#include <common/logger_useful.h>
@ -146,7 +147,7 @@ struct AggregateFunctionSessionSplitData
<< "Sorted " << std::to_string(size) << " rows SessionEvent data."
<< " in " << elapsed << " sec.";
LOG_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), log_helper.str());
LOG_TRACE(getLogger(__PRETTY_FUNCTION__), log_helper.str());
sorted = true;
}
@ -195,7 +196,7 @@ struct AggregateFunctionSessionSplitData
<< " in " << elapsed << " sec."
<< " (" << other.events.size() / elapsed << " rows/sec.)";
LOG_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), log_helper.str());
LOG_TRACE(getLogger(__PRETTY_FUNCTION__), log_helper.str());
}
void serialize(WriteBuffer & buf) const

View File

@ -15,6 +15,7 @@
#pragma once
#include <Common/Logger.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Analyzers/ASTEquals.h>
#include <Analyzers/ResolvedWindow.h>
@ -243,7 +244,7 @@ using ListMultimap = std::unordered_map<Key, std::vector<Val>>;
struct Analysis
{
ScopeFactory scope_factory;
Poco::Logger * logger = &Poco::Logger::get("Analysis");
LoggerPtr logger = getLogger("Analysis");
/// Scopes
// Regular scopes in an ASTSelectQuery, kept by below convention:

View File

@ -145,7 +145,7 @@ private:
const bool enable_subcolumn_optimization_through_union;
const bool enable_implicit_arg_type_convert; // MySQL implicit cast rules
Poco::Logger * logger = &Poco::Logger::get("QueryAnalyzerVisitor");
LoggerPtr logger = getLogger("QueryAnalyzerVisitor");
void analyzeSetOperation(ASTPtr & node, ASTs & selects);

View File

@ -548,7 +548,7 @@ namespace
ASTPtr QueryRewriter::rewrite(ASTPtr query, ContextMutablePtr context, bool enable_materialized_view)
{
const auto * logger = &Poco::Logger::get("QueryRewriter");
const auto logger = getLogger("QueryRewriter");
(void) enable_materialized_view;
graphviz_index = GraphvizPrinter::PRINT_AST_INDEX;

View File

@ -295,7 +295,7 @@ void RewriteFusionMerge::visit(ASTTableExpression & table_expr, ASTPtr &)
table_expr.children.push_back(table_expr.subquery);
LOG_DEBUG(
&Poco::Logger::get("RewriteFusionMerge"), "Rewrite {} to {}", serializeAST(*table_func_ptr), serializeAST(*select_union_query));
getLogger("RewriteFusionMerge"), "Rewrite {} to {}", serializeAST(*table_func_ptr), serializeAST(*select_union_query));
}
}

View File

@ -54,7 +54,7 @@ namespace ErrorCodes
namespace
{
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log)
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, LoggerPtr log)
{
Poco::Net::SocketAddress socket_address;
try
@ -78,7 +78,7 @@ namespace
return socket_address;
}
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, Poco::Logger * log)
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, LoggerPtr log)
{
auto address = makeSocketAddress(host, port, log);
#if POCO_VERSION < 0x01080000
@ -190,7 +190,7 @@ void IBridge::initialize(Application & self)
BaseDaemon::logRevision();
log = &logger();
log = getLogger(logger());
hostname = config().getString("listen-host", "127.0.0.1");
port = config().getUInt("http-port");
if (port > 0xFFFF)

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <daemon/BaseDaemon.h>
@ -46,6 +47,6 @@ private:
size_t max_server_connections;
size_t http_timeout;
Poco::Logger * log;
LoggerPtr log;
};
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context.h>
#include <Poco/Logger.h>
#include <Poco/Util/AbstractConfiguration.h>
@ -54,7 +55,7 @@ protected:
virtual const Poco::Util::AbstractConfiguration & getConfig() const = 0;
virtual Poco::Logger * getLog() const = 0;
virtual LoggerPtr getLog() const = 0;
virtual Poco::Timespan getHTTPTimeout() const = 0;

View File

@ -8,7 +8,7 @@ namespace DB
LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_)
: IBridgeHelper(context_)
, config(context_->getConfigRef())
, log(&Poco::Logger::get("LibraryBridgeHelper"))
, log(getLogger("LibraryBridgeHelper"))
, http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value)
, bridge_host(config.getString("library_bridge.host", DEFAULT_HOST))
, bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT))

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Poco/Logger.h>
@ -31,7 +32,7 @@ protected:
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
Poco::Logger * getLog() const override { return log; }
LoggerPtr getLog() const override { return log; }
Poco::Timespan getHTTPTimeout() const override { return http_timeout; }
@ -40,7 +41,7 @@ protected:
static constexpr inline size_t DEFAULT_PORT = 9012;
const Poco::Util::AbstractConfiguration & config;
Poco::Logger * log;
LoggerPtr log;
const Poco::Timespan http_timeout;
std::string bridge_host;
size_t bridge_port;

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Interpreters/Context.h>
@ -67,7 +68,7 @@ public:
Poco::Timespan http_timeout_,
const std::string & connection_string_)
: IXDBCBridgeHelper(context_->getGlobalContext())
, log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
, log(getLogger(BridgeHelperMixin::getName() + "BridgeHelper"))
, connection_string(connection_string_)
, http_timeout(http_timeout_)
, config(context_->getGlobalContext()->getConfigRef())
@ -122,7 +123,7 @@ protected:
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
Poco::Logger * getLog() const override { return log; }
LoggerPtr getLog() const override { return log; }
bool startBridgeManually() const override { return BridgeHelperMixin::startBridgeManually(); }
@ -144,7 +145,7 @@ protected:
private:
using Configuration = Poco::Util::AbstractConfiguration;
Poco::Logger * log;
LoggerPtr log;
std::string connection_string;
Poco::Timespan http_timeout;
std::string bridge_host;

View File

@ -4096,7 +4096,7 @@ namespace Catalog
return txn_undobuffers;
}
Catalog::UndoBufferIterator::UndoBufferIterator(IMetaStore::IteratorPtr metastore_iter_, Poco::Logger * log_)
Catalog::UndoBufferIterator::UndoBufferIterator(IMetaStore::IteratorPtr metastore_iter_, LoggerPtr log_)
: metastore_iter{std::move(metastore_iter_)}, log{log_}
{}
@ -7340,7 +7340,7 @@ namespace Catalog
void notifyOtherServersOnAccessEntityChange(const Context & context, EntityType type, const String & name, const UUID & uuid)
{
static Poco::Logger * log = &Poco::Logger::get("Catalog::notifyOtherServersOnAccessEntityChange");
static LoggerPtr log = getLogger("Catalog::notifyOtherServersOnAccessEntityChange");
std::shared_ptr<CnchTopologyMaster> topology_master = context.getCnchTopologyMaster();
if (!topology_master)
{
@ -7770,7 +7770,7 @@ namespace Catalog
}
catch (...)
{
tryLogCurrentException(&Poco::Logger::get("Catalog::getLastModificationTimeHints"));
tryLogCurrentException(getLogger("Catalog::getLastModificationTimeHints"));
}
}

View File

@ -15,6 +15,7 @@
#pragma once
#include <Common/Logger.h>
#include <atomic>
#include <map>
#include <optional>
@ -506,7 +507,7 @@ public:
class UndoBufferIterator
{
public:
UndoBufferIterator(IMetaStore::IteratorPtr metastore_iter, Poco::Logger * log);
UndoBufferIterator(IMetaStore::IteratorPtr metastore_iter, LoggerPtr log);
const UndoResource & getUndoResource() const;
bool next();
bool is_valid() const /// for testing
@ -517,7 +518,7 @@ public:
IMetaStore::IteratorPtr metastore_iter;
std::optional<UndoResource> cur_undo_resource;
bool valid = false;
Poco::Logger * log;
LoggerPtr log;
};
UndoBufferIterator getUndoBufferIterator() const;
@ -918,7 +919,7 @@ public:
void shutDown() {bg_task.reset();}
private:
Poco::Logger * log = &Poco::Logger::get("Catalog");
LoggerPtr log = getLogger("Catalog");
Context & context;
MetastoreProxyPtr meta_proxy;
const String name_space;

View File

@ -1,6 +1,7 @@
#pragma once
#include <Catalog/IMetastore.h>
#include <Common/Logger.h>
#include <Core/BackgroundSchedulePool.h>
#include <Interpreters/Context.h>
@ -27,7 +28,7 @@ private:
void cleanStaleLargeKV();
Poco::Logger * log = &Poco::Logger::get("CatalogBGTask");
LoggerPtr log = getLogger("CatalogBGTask");
ContextPtr context;
std::shared_ptr<IMetaStore> metastore;

View File

@ -125,7 +125,7 @@ ASTPtr CatalogFactory::getCreateDictionaryByDataModel(const DB::Protos::DataMode
}
catch (Exception &)
{
LOG_WARNING(&Poco::Logger::get("CatalogFactory"), "Dictionary create query parse failed: query {}", create_query);
LOG_WARNING(getLogger("CatalogFactory"), "Dictionary create query parse failed: query {}", create_query);
throw;
}

View File

@ -77,7 +77,7 @@ static fdb_error_t RunWithRetry(FDBTransactionPtr tr, size_t max_retry, Runnable
if (fdb_error_t f_code = waitFuture(f->future); f_code)
return code;
// continue the loop and perform the operation again.
LOG_WARNING(&Poco::Logger::get("FDBClient::RunWithRetry"), "Try perform the transaction again with retryable error : {}, remain retry time: {}",
LOG_WARNING(getLogger("FDBClient::RunWithRetry"), "Try perform the transaction again with retryable error : {}, remain retry time: {}",
std::string(fdb_get_error(code)), max_retry);
}
}
@ -478,7 +478,7 @@ bool Iterator::Next(fdb_error_t & code)
if (code == FDBError::FDB_transaction_too_old
|| code == FDBError::FDB_transaction_timed_out)
{
LOG_DEBUG(&Poco::Logger::get("FDBIterator"), "Transaction timeout or too old, create new transaction");
LOG_DEBUG(getLogger("FDBIterator"), "Transaction timeout or too old, create new transaction");
tr = std::make_shared<FDB::FDBTransactionRAII>();
Catalog::MetastoreFDBImpl::check_fdb_op(client->CreateTransaction(tr));
continue;

View File

@ -553,7 +553,7 @@ String MetastoreProxy::getMvMetaVersion(const String & name_space, const String
{
String mv_meta_version_ts;
metastore_ptr->get(matViewVersionKey(name_space, uuid), mv_meta_version_ts);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "get mv meta, version {}.", mv_meta_version_ts);
LOG_TRACE(getLogger("MetaStore"), "get mv meta, version {}.", mv_meta_version_ts);
if (mv_meta_version_ts.empty())
return "";
@ -565,7 +565,7 @@ BatchCommitRequest MetastoreProxy::constructMvMetaRequests(const String & name_s
std::vector<std::shared_ptr<Protos::VersionedPartition>> drop_partitions,
String mv_version_ts)
{
LOG_TRACE(&Poco::Logger::get("MetaStore"), "construct mv meta, version {}.", mv_version_ts);
LOG_TRACE(getLogger("MetaStore"), "construct mv meta, version {}.", mv_version_ts);
BatchCommitRequest multi_write;
for (const auto & add : add_partitions)
@ -575,7 +575,7 @@ BatchCommitRequest MetastoreProxy::constructMvMetaRequests(const String & name_s
add->SerializeToString(&value);
String key = matViewBaseTablesKey(name_space, uuid, base_uuid, add->partition());
multi_write.AddPut(SinglePutRequest(key, value));
LOG_TRACE(&Poco::Logger::get("MetaStore"), "add key {} value size {}.", key, value.size());
LOG_TRACE(getLogger("MetaStore"), "add key {} value size {}.", key, value.size());
}
multi_write.AddPut(SinglePutRequest(matViewVersionKey(name_space, uuid), mv_version_ts));
@ -587,7 +587,7 @@ BatchCommitRequest MetastoreProxy::constructMvMetaRequests(const String & name_s
drop->SerializeToString(&value);
String key = matViewBaseTablesKey(name_space, uuid, base_uuid, drop->partition());
multi_write.AddDelete(SinglePutRequest(key, value));
LOG_TRACE(&Poco::Logger::get("MetaStore"), "drop key {}.", key);
LOG_TRACE(getLogger("MetaStore"), "drop key {}.", key);
}
return multi_write;
@ -605,7 +605,7 @@ void MetastoreProxy::updateMvMeta(const String & name_space, const String & uuid
String serialized_meta;
p.SerializeToString(&serialized_meta);
metastore_ptr->put(matViewBaseTablesKey(name_space, uuid, base_uuid, p.partition()), serialized_meta);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "value size {}.", serialized_meta.size());
LOG_TRACE(getLogger("MetaStore"), "value size {}.", serialized_meta.size());
}
}
}
@ -1188,7 +1188,7 @@ void MetastoreProxy::createMutation(const String & name_space, const String & uu
void MetastoreProxy::removeMutation(const String & name_space, const String & uuid, const String & mutation_name)
{
LOG_TRACE(&Poco::Logger::get(__func__), "Removing mutation {}", mutation_name);
LOG_TRACE(getLogger(__func__), "Removing mutation {}", mutation_name);
metastore_ptr->drop(tableMutationKey(name_space, uuid, mutation_name));
}
@ -1425,7 +1425,7 @@ void MetastoreProxy::clearIntents(const String & name_space, const String & inte
auto snapshot = metastore_ptr->multiGet(intent_names);
Poco::Logger * log = &Poco::Logger::get(__func__);
LoggerPtr log = getLogger(__func__);
std::vector<size_t> matched_intent_index;
for (size_t i = 0; i < intents.size(); i++)
@ -2816,7 +2816,7 @@ void MetastoreProxy::attachDetachedParts(
{
auto info_ptr = createPartInfoFromModel(parts.parts(idx).part_info());
String part_key = dataPartKey(name_space, to_uuid, info_ptr->getPartName());
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[attachDetachedParts] Write part record {}", part_key);
LOG_TRACE(getLogger("MetaStore"), "[attachDetachedParts] Write part record {}", part_key);
if (!existing_partitions.contains(info_ptr->partition_id) && !partition_map.contains(info_ptr->partition_id))
{
@ -2829,7 +2829,7 @@ void MetastoreProxy::attachDetachedParts(
{
auto info_ptr = createPartInfoFromModel(staged_parts.parts(idx).part_info());
String staged_part_key = stagedDataPartKey(name_space, to_uuid, info_ptr->getPartName());
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[attachDetachedStagedParts] Write part record {}", staged_part_key);
LOG_TRACE(getLogger("MetaStore"), "[attachDetachedStagedParts] Write part record {}", staged_part_key);
if (!existing_partitions.contains(info_ptr->partition_id) && !partition_map.contains(info_ptr->partition_id))
{
@ -2845,7 +2845,7 @@ void MetastoreProxy::attachDetachedParts(
partition_model.set_id(partition_id);
partition_model.set_partition_minmax(partition_minmax);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[attachDetachedParts] Write partition record {}",
LOG_TRACE(getLogger("MetaStore"), "[attachDetachedParts] Write partition record {}",
partition_key);
batch_writer.addPut(partition_key, partition_model.SerializeAsString());
@ -2864,7 +2864,7 @@ void MetastoreProxy::attachDetachedParts(
{
String detached_part_key = detachedPartKey(name_space, from_uuid,
detached_part_names[idx]);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[attachDetachedParts] Delete detached part record {}",
LOG_TRACE(getLogger("MetaStore"), "[attachDetachedParts] Delete detached part record {}",
detached_part_key);
batch_writer.addDelete(detached_part_key);
@ -2880,7 +2880,7 @@ void MetastoreProxy::attachDetachedParts(
{
const auto & bitmap_model_meta = bitmap_meta->getModel();
String detached_bitmap_meta_key = deleteBitmapKey(name_space, to_uuid, *bitmap_model_meta);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[attachDetachedDeleteBitmaps] Write new bitmap meta record {}", detached_bitmap_meta_key);
LOG_TRACE(getLogger("MetaStore"), "[attachDetachedDeleteBitmaps] Write new bitmap meta record {}", detached_bitmap_meta_key);
batch_writer.addPut(detached_bitmap_meta_key, bitmap_model_meta->SerializeAsString());
}
@ -2893,7 +2893,7 @@ void MetastoreProxy::attachDetachedParts(
for (auto & bitmap_meta: detached_bitmaps)
{
String detached_bitmap_meta_key = detachedDeleteBitmapKey(name_space, from_uuid, *bitmap_meta->getModel());
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[detachAttachedDeleteBitmaps] Delete detached bitmap meta record {}", detached_bitmap_meta_key);
LOG_TRACE(getLogger("MetaStore"), "[detachAttachedDeleteBitmaps] Delete detached bitmap meta record {}", detached_bitmap_meta_key);
batch_writer.addDelete(detached_bitmap_meta_key);
}
@ -2937,7 +2937,7 @@ void MetastoreProxy::detachAttachedParts(
auto info_ptr = createPartInfoFromModel(parts[idx].value().part_info());
String detached_part_key = detachedPartKey(name_space, to_uuid,
info_ptr->getPartName());
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[detachAttachedParts] Write detach part record {}",
LOG_TRACE(getLogger("MetaStore"), "[detachAttachedParts] Write detach part record {}",
detached_part_key);
batch_writer.addPut(detached_part_key, parts[idx].value().SerializeAsString());
@ -2952,7 +2952,7 @@ void MetastoreProxy::detachAttachedParts(
for (size_t idx = 0; idx < attached_part_names.size(); ++idx)
{
String part_key = dataPartKey(name_space, from_uuid, attached_part_names[idx]);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[detachAttachedParts] Delete part record {}",
LOG_TRACE(getLogger("MetaStore"), "[detachAttachedParts] Delete part record {}",
part_key);
batch_writer.addDelete(part_key);
@ -2960,7 +2960,7 @@ void MetastoreProxy::detachAttachedParts(
for (size_t idx = 0; idx < attached_staged_part_names.size(); ++idx)
{
String part_key = stagedDataPartKey(name_space, from_uuid, attached_staged_part_names[idx]);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[detachAttachedParts] Delete staged part record {}", part_key);
LOG_TRACE(getLogger("MetaStore"), "[detachAttachedParts] Delete staged part record {}", part_key);
batch_writer.addDelete(part_key);
}
@ -2976,7 +2976,7 @@ void MetastoreProxy::detachAttachedParts(
const auto & bitmap_model_meta = bitmap_meta->getModel();
String detached_bitmap_meta_key = detachedDeleteBitmapKey(name_space, to_uuid, *bitmap_model_meta);
LOG_TRACE(
&Poco::Logger::get("MetaStore"),
getLogger("MetaStore"),
"[detachAttachedDeleteBitmaps] Write detach bitmap meta record {}",
detached_bitmap_meta_key);
@ -2992,7 +2992,7 @@ void MetastoreProxy::detachAttachedParts(
{
String detached_bitmap_meta_key = deleteBitmapKey(name_space, from_uuid, *bitmap_meta->getModel());
LOG_TRACE(
&Poco::Logger::get("MetaStore"), "[detachAttachedDeleteBitmaps] Delete bitmap meta record {}", detached_bitmap_meta_key);
getLogger("MetaStore"), "[detachAttachedDeleteBitmaps] Delete bitmap meta record {}", detached_bitmap_meta_key);
batch_writer.addDelete(detached_bitmap_meta_key);
}
@ -3044,7 +3044,7 @@ std::vector<std::pair<String, UInt64>> MetastoreProxy::attachDetachedPartsRaw(
else
part_key = stagedDataPartKey(name_space, tbl_uuid, part_names[i]);
LOG_TRACE(
&Poco::Logger::get("MetaStore"),
getLogger("MetaStore"),
"[attachDetachedPartsRaw] Write {} part meta record {}",
i < detached_visible_part_size ? "" : "staged ",
part_key);
@ -3061,7 +3061,7 @@ std::vector<std::pair<String, UInt64>> MetastoreProxy::attachDetachedPartsRaw(
for (size_t i = 0; i < part_names.size(); ++i)
{
String detached_part_key = detachedPartKey(name_space, tbl_uuid, part_names[i]);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "[attachDetachedPartsRaw] Delete detached part record {}",
LOG_TRACE(getLogger("MetaStore"), "[attachDetachedPartsRaw] Delete detached part record {}",
detached_part_key);
batch_writer.addDelete(detached_part_key);
@ -3075,7 +3075,7 @@ std::vector<std::pair<String, UInt64>> MetastoreProxy::attachDetachedPartsRaw(
for (size_t i = 0; i < bitmap_names.size(); ++i)
{
String detached_bitmap_meta_key = detachedDeleteBitmapKey(name_space, tbl_uuid, bitmap_names[i]);
LOG_TRACE(&Poco::Logger::get("MS"), "[attachDetachedPartsRaw] Delete detached bitmap meta record {}", detached_bitmap_meta_key);
LOG_TRACE(getLogger("MS"), "[attachDetachedPartsRaw] Delete detached bitmap meta record {}", detached_bitmap_meta_key);
batch_writer.addDelete(detached_bitmap_meta_key);
}
@ -3102,7 +3102,7 @@ void MetastoreProxy::detachAttachedPartsRaw(
for (const auto& [detached_part_name, detached_part_meta] : detached_part_metas)
{
String detached_part_key = detachedPartKey(name_space, to_uuid, detached_part_name);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "Write detached part record {} in detachAttachedPartsRaw",
LOG_TRACE(getLogger("MetaStore"), "Write detached part record {} in detachAttachedPartsRaw",
detached_part_key);
batch_writer.addPut(detached_part_key, detached_part_meta);
@ -3117,11 +3117,11 @@ void MetastoreProxy::detachAttachedPartsRaw(
{
/// We don't know whether attach a staged part or normal part, just delete both.
String attached_part_key = dataPartKey(name_space, from_uuid, attached_part_name);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "Delete part record {} in detachAttachedPartsRaw",
LOG_TRACE(getLogger("MetaStore"), "Delete part record {} in detachAttachedPartsRaw",
attached_part_key);
String attached_staged_part_key = stagedDataPartKey(name_space, from_uuid, attached_part_name);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "Delete staged part record {} in detachAttachedPartsRaw", attached_staged_part_key);
LOG_TRACE(getLogger("MetaStore"), "Delete staged part record {} in detachAttachedPartsRaw", attached_staged_part_key);
batch_writer.addDelete(attached_part_key);
batch_writer.addDelete(attached_staged_part_key);
@ -3135,7 +3135,7 @@ void MetastoreProxy::detachAttachedPartsRaw(
for (const auto & [detached_bitmap_name, detached_bitmap_meta] : detached_bitmap_metas)
{
String detached_bitmap_key = detachedDeleteBitmapKey(name_space, to_uuid, detached_bitmap_name);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "Write detached bitmap record {} in detachAttachedPartsRaw", detached_bitmap_key);
LOG_TRACE(getLogger("MetaStore"), "Write detached bitmap record {} in detachAttachedPartsRaw", detached_bitmap_key);
batch_writer.addPut(detached_bitmap_key, detached_bitmap_meta);
}
@ -3148,7 +3148,7 @@ void MetastoreProxy::detachAttachedPartsRaw(
for (const String & attached_bitmap_name : attached_bitmap_names)
{
String attached_bitmap_key = deleteBitmapKey(name_space, from_uuid, attached_bitmap_name);
LOG_TRACE(&Poco::Logger::get("MetaStore"), "Delete bitmap record {} in detachAttachedPartsRaw", attached_bitmap_key);
LOG_TRACE(getLogger("MetaStore"), "Delete bitmap record {} in detachAttachedPartsRaw", attached_bitmap_key);
batch_writer.addDelete(attached_bitmap_key);
}
@ -3244,7 +3244,7 @@ bool MetastoreProxy::resetObjectAssembledSchemaAndPurgePartialSchemas(
const SerializedObjectSchema & new_assembled_schema,
const std::vector<TxnTimestamp> & partial_schema_txnids)
{
Poco::Logger * log = &Poco::Logger::get(__func__);
LoggerPtr log = getLogger(__func__);
BatchCommitRequest batch_write;
bool if_not_exists = false;
@ -3348,7 +3348,7 @@ Strings MetastoreProxy::removePartitions(const String & name_space, const String
auto partitions_meta = metastore_ptr->multiGet(request_keys);
Poco::Logger * log = &Poco::Logger::get(__func__);
LoggerPtr log = getLogger(__func__);
Strings res;
// try commit all partitions with CAS in one batch

View File

@ -15,6 +15,7 @@
#pragma once
#include <Catalog/HandlerManager.h>
#include <Common/Logger.h>
#include <Core/Types.h>
#include <Poco/Logger.h>
#include <brpc/stream.h>
@ -38,7 +39,7 @@ public:
virtual void on_closed(brpc::StreamId) override;
Poco::Logger * log = &Poco::Logger::get("StreamingHandler");
LoggerPtr log = getLogger("StreamingHandler");
HandlerManager & manager;
HandlerIterator handler_it;
};

View File

@ -21,6 +21,7 @@
#pragma once
#include <Common/Logger.h>
#include "common/types.h"
#include <common/logger_useful.h>
@ -28,6 +29,7 @@
#include <Common/HostWithPorts.h>
#include <Common/Throttler.h>
#include <Common/callOnce.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
@ -364,16 +366,17 @@ private:
{
}
Poco::Logger * get()
LoggerPtr get()
{
if (!log)
log = &Poco::Logger::get("Connection (" + parent.getDescription() + ")");
callOnce(log_initialized, [&] {
log = getLogger("Connection (" + parent.getDescription() + ")");
});
return log;
}
private:
std::atomic<Poco::Logger *> log;
mutable OnceFlag log_initialized;
LoggerPtr log;
Connection & parent;
};

View File

@ -22,7 +22,7 @@ ConnectionEstablisher::ConnectionEstablisher(
IConnectionPool * pool_,
const ConnectionTimeouts * timeouts_,
const Settings * settings_,
Poco::Logger * log_,
LoggerPtr log_,
const QualifiedTableName * table_to_check_)
: pool(pool_), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_), is_finished(false)
{
@ -109,7 +109,7 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync(
IConnectionPool * pool_,
const ConnectionTimeouts * timeouts_,
const Settings * settings_,
Poco::Logger * log_,
LoggerPtr log_,
const QualifiedTableName * table_to_check_)
: connection_establisher(pool_, timeouts_, settings_, log_, table_to_check_)
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <variant>
#include <Common/Epoll.h>
@ -22,7 +23,7 @@ public:
ConnectionEstablisher(IConnectionPool * pool_,
const ConnectionTimeouts * timeouts_,
const Settings * settings_,
Poco::Logger * log,
LoggerPtr log,
const QualifiedTableName * table_to_check = nullptr);
/// Establish connection and save it in result, write possible exception message in fail_message.
@ -37,7 +38,7 @@ private:
IConnectionPool * pool;
const ConnectionTimeouts * timeouts;
const Settings * settings;
Poco::Logger * log;
LoggerPtr log;
const QualifiedTableName * table_to_check;
bool is_finished;
@ -61,7 +62,7 @@ public:
ConnectionEstablisherAsync(IConnectionPool * pool_,
const ConnectionTimeouts * timeouts_,
const Settings * settings_,
Poco::Logger * log_,
LoggerPtr log_,
const QualifiedTableName * table_to_check = nullptr);
/// Resume establishing connection. If the process was not finished,

View File

@ -21,6 +21,7 @@
#pragma once
#include <Common/Logger.h>
#include <Common/PoolBase.h>
#include <Client/Connection.h>
#include <IO/ConnectionTimeouts.h>
@ -85,7 +86,7 @@ public:
UInt16 rpc_port_ = 0,
String worker_id_ = "virtual_id")
: Base(max_connections_,
&Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
getLogger("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
host(host_),
port(port_),
default_database(default_database_),

View File

@ -29,7 +29,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover(
LoadBalancing load_balancing,
time_t decrease_error_period_,
size_t max_error_cap_)
: Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover"))
: Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, getLogger("ConnectionPoolWithFailover"))
, default_load_balancing(load_balancing)
{
const std::string & local_hostname = getFQDNOrHostName();

View File

@ -24,7 +24,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory(
const Settings * settings_,
const ConnectionTimeouts & timeouts_,
std::shared_ptr<QualifiedTableName> table_to_check_)
: pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_), log(&Poco::Logger::get("HedgedConnectionsFactory"))
: pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_), log(getLogger("HedgedConnectionsFactory"))
{
shuffled_pools = pool->getShuffledPools(settings);
for (auto shuffled_pool : shuffled_pools)

View File

@ -2,6 +2,7 @@
#if defined(OS_LINUX)
#include <Common/Logger.h>
#include <Common/TimerDescriptor.h>
#include <Common/Epoll.h>
#include <Common/FiberStack.h>
@ -129,7 +130,7 @@ private:
int last_used_index = -1;
bool fallback_to_stale_replicas;
Epoll epoll;
Poco::Logger * log;
LoggerPtr log;
std::string fail_messages;
/// The maximum number of attempts to connect to replicas.

View File

@ -52,7 +52,7 @@ CloudMergeTreeDedupWorker::CloudMergeTreeDedupWorker(StorageCloudMergeTree & sto
: storage(storage_)
, context(storage.getContext())
, log_name(storage.getLogName() + "(DedupWorker)")
, log(&Poco::Logger::get(log_name))
, log(getLogger(log_name))
, interval_scheduler(storage.getSettings()->staged_part_lifetime_threshold_ms_to_block_kafka_consume)
{
task = storage.getContext()->getUniqueTableSchedulePool().createTask(log_name, [this] { run(); });

View File

@ -15,6 +15,7 @@
#pragma once
#include <Common/Logger.h>
#include <CloudServices/DedupWorkerStatus.h>
#include <CloudServices/DedupGran.h>
#include <Core/BackgroundSchedulePool.h>
@ -128,7 +129,7 @@ private:
else /// idle area
ratio = 1.5;
LOG_DEBUG(
&Poco::Logger::get("TaskIntervalScheduler"),
getLogger("TaskIntervalScheduler"),
"min staged part timestamp: {} ms, current timestamp: {} ms, final ratio is: {}, current sleep time: {} ms.",
mts,
current_timestamp.toMillisecond(),
@ -154,7 +155,7 @@ private:
std::atomic<size_t> index{0};
ContextMutablePtr context;
String log_name;
Poco::Logger * log;
LoggerPtr log;
BackgroundSchedulePool::TaskHolder task;
TaskIntervalScheduler interval_scheduler;
std::atomic<bool> is_stopped{false};

View File

@ -26,7 +26,7 @@ HAVING insert_parts > 0;
)";
CnchBGThreadPartitionSelector::CnchBGThreadPartitionSelector(ContextMutablePtr global_context_)
: WithMutableContext(global_context_), log(&Poco::Logger::get("PartitionSelector"))
: WithMutableContext(global_context_), log(getLogger("PartitionSelector"))
{
try
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <Common/Logger.h>
#include <Core/Types.h>
#include <Interpreters/Context_fwd.h>
#include <Poco/Logger.h>
@ -132,7 +133,7 @@ private:
bool needRoundRobinPick(const StoragePtr & storage, Type type, size_t & out_n_suggestion);
Strings doRoundRobinPick(const StoragePtr & storage, Type type, size_t n);
Poco::Logger * log;
LoggerPtr log;
/// Whether loading digest information from system.server_part_log successful.
bool load_success = true;

View File

@ -153,7 +153,7 @@ CnchBGThreadPtr CnchBGThreadsMap::startThread(const StorageID & storage_id)
{
// Create new MergeThread but not start it,
// to prevent daemon_manager send duplicate startMergeThread request
auto * log = &Poco::Logger::get("CnchBGThreadsMap");
auto log = getLogger("CnchBGThreadsMap");
LOG_DEBUG(log, "Cancel start MergeThread for table {}, since table on the blacklist.", storage_id.getNameForLogs());
}
else
@ -171,7 +171,7 @@ void CnchBGThreadsMap::tryRemoveThread(const StorageID & storage_id)
auto t = tryGetThread(storage_id);
if (!t)
{
LOG_DEBUG(&Poco::Logger::get("CnchBGThreadsMap"), "{} for {} not found", toString(type), storage_id.getNameForLogs());
LOG_DEBUG(getLogger("CnchBGThreadsMap"), "{} for {} not found", toString(type), storage_id.getNameForLogs());
return;
}
@ -184,7 +184,7 @@ void CnchBGThreadsMap::tryDropThread(const StorageID & storage_id)
auto t = tryGetThread(storage_id);
if (!t)
{
LOG_DEBUG(&Poco::Logger::get("CnchBGThreadsMap"), "{} for {} not found", toString(type), storage_id.getNameForLogs());
LOG_DEBUG(getLogger("CnchBGThreadsMap"), "{} for {} not found", toString(type), storage_id.getNameForLogs());
return;
}
@ -222,7 +222,7 @@ void CnchBGThreadsMap::cleanup()
if (it->second->error())
{
LOG_WARNING(
&Poco::Logger::get("CnchBGThreadsMap"),
getLogger("CnchBGThreadsMap"),
"{} for {} got error, remove it",
toString(type),
it->second->getStorageID().getNameForLogs());

View File

@ -544,7 +544,7 @@ void CnchDataWriter::commitPreparedCnchParts(const DumpedData & dumped_data, con
if (context->getServerType() != ServerType::cnch_server)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Must be called in Server mode: {}", context->getServerType());
auto * log = storage.getLogger();
auto log = storage.getLogger();
auto txn = context->getCurrentTransaction();
auto txn_id = txn->getTransactionID();
/// set main table uuid in server side

View File

@ -489,9 +489,9 @@ namespace
if (!partition_sorted || !partition_aligned)
{
if (!partition_sorted)
LOG_WARNING(&Poco::Logger::get(__func__), "parts are not partition sorted, this could make calcVisible slow");
LOG_WARNING(getLogger(__func__), "parts are not partition sorted, this could make calcVisible slow");
else if (partition_ids.size() > 1)
LOG_WARNING(&Poco::Logger::get(__func__), "parts are not partition aligned, this could make calcVisible slow");
LOG_WARNING(getLogger(__func__), "parts are not partition aligned, this could make calcVisible slow");
process_parts(all_parts, 0, all_parts.size(), visible_parts);
}
else
@ -531,7 +531,7 @@ namespace
if (logging == EnableLogging)
{
auto log = &Poco::Logger::get(__func__);
auto log = getLogger(__func__);
LOG_DEBUG(log, "all_parts:\n {}", partsToDebugString(all_parts));
LOG_DEBUG(log, "visible_parts (skip_drop_ranges={}):\n{}", skip_drop_ranges, partsToDebugString(visible_parts));
if (visible_alone_drop_ranges)

View File

@ -673,7 +673,7 @@ void CnchServerClient::precommitParts(
size_t staged_part_batch_begin = std::min(batch_begin, staged_parts.size());
size_t staged_part_batch_end = std::min(batch_end, staged_parts.size());
Poco::Logger * log = &Poco::Logger::get(__func__);
LoggerPtr log = getLogger(__func__);
LOG_DEBUG(
log,
"Precommit: parts in batch: [{} ~ {}] of total: {}; delete_bitmaps in batch [{} ~ {}] of total {}; staged parts in batch [{} "
@ -783,7 +783,7 @@ void CnchServerClient::cleanTransaction(const TransactionRecord & txn_record)
Protos::CleanTransactionReq request;
Protos::CleanTransactionResp response;
LOG_DEBUG(&Poco::Logger::get(__func__), "clean txn: [{}] on server: {}", txn_record.toString(), getRPCAddress());
LOG_DEBUG(getLogger(__func__), "clean txn: [{}] on server: {}", txn_record.toString(), getRPCAddress());
request.mutable_txn_record()->CopyFrom(txn_record.pb_model);
stub->cleanTransaction(&cntl, &request, &response, nullptr);
@ -798,7 +798,7 @@ void CnchServerClient::cleanUndoBuffers(const TransactionRecord & txn_record)
Protos::CleanUndoBuffersReq request;
Protos::CleanUndoBuffersResp response;
LOG_DEBUG(&Poco::Logger::get(__func__), "clean undo buffers for txn: [{}] on server: {}", txn_record.toString(), getRPCAddress());
LOG_DEBUG(getLogger(__func__), "clean undo buffers for txn: [{}] on server: {}", txn_record.toString(), getRPCAddress());
request.mutable_txn_record()->CopyFrom(txn_record.pb_model);
stub->cleanUndoBuffers(&cntl, &request, &response, nullptr);

View File

@ -14,6 +14,7 @@
*/
#pragma once
#include <Common/Logger.h>
#include <optional>
#include <unordered_map>
#include <unordered_set>
@ -139,7 +140,7 @@ class CnchServerResource
{
public:
explicit CnchServerResource(TxnTimestamp curr_txn_id)
: txn_id(curr_txn_id), log(&Poco::Logger::get("ServerResource"))
: txn_id(curr_txn_id), log(getLogger("ServerResource"))
{
}
@ -280,7 +281,7 @@ private:
ResourceStageInfo resource_stage_info;
bool skip_clean_worker{false};
Poco::Logger * log;
LoggerPtr log;
mutable ServerResourceLockManager lock_manager;
bool send_mutations{false};

View File

@ -111,7 +111,7 @@ CnchServerServiceImpl::CnchServerServiceImpl(ContextMutablePtr global_context)
: WithMutableContext(global_context),
server_start_time(getTS(global_context)),
global_gc_manager(global_context),
log(&Poco::Logger::get("CnchServerService"))
log(getLogger("CnchServerService"))
{
}
@ -177,7 +177,7 @@ void CnchServerServiceImpl::commitParts(
auto column_commit_time = storage->getPartColumnsCommitTime(*(parts[0]->getColumnsPtr()));
if (column_commit_time != storage->commit_time.toUInt64())
{
LOG_WARNING(&Poco::Logger::get("CnchServerService"), "Kafka consumer cannot commit parts because of underlying table change. Will reschedule consume task.");
LOG_WARNING(getLogger("CnchServerService"), "Kafka consumer cannot commit parts because of underlying table change. Will reschedule consume task.");
throw Exception(ErrorCodes::CNCH_KAFKA_TASK_NEED_STOP, "Commit fails because of storage schema change");
}
}
@ -187,7 +187,7 @@ void CnchServerServiceImpl::commitParts(
for (const auto & tp : req->tpl())
tpl.emplace_back(cppkafka::TopicPartition(tp.topic(), tp.partition(), tp.offset()));
LOG_TRACE(&Poco::Logger::get("CnchServerService"), "parsed tpl to commit with size: {}\n", tpl.size());
LOG_TRACE(getLogger("CnchServerService"), "parsed tpl to commit with size: {}\n", tpl.size());
}
MySQLBinLogInfo binlog;

View File

@ -19,6 +19,7 @@
# include "config_core.h"
#endif
#include <Common/Logger.h>
#include <MergeTreeCommon/GlobalGCManager.h>
#include <Interpreters/Context_fwd.h>
#include <Protos/cnch_server_rpc.pb.h>
@ -388,7 +389,7 @@ public:
private:
const UInt64 server_start_time;
std::optional<GlobalGCManager> global_gc_manager;
Poco::Logger * log;
LoggerPtr log;
};
REGISTER_SERVICE_IMPL(CnchServerServiceImpl);

View File

@ -44,7 +44,7 @@ void CnchWorkerClientPools::addVirtualWarehouseImpl(const String & name, const S
/// Will replace old pool
pools[name] = std::move(pool);
LOG_INFO(&Poco::Logger::get("CnchWorkerClientPools"), "Added new vw: {} ", name);
LOG_INFO(getLogger("CnchWorkerClientPools"), "Added new vw: {} ", name);
}
void CnchWorkerClientPools::removeVirtualWarehouse(const String & name)

View File

@ -55,7 +55,7 @@ static ASTPtr parseCreateQuery(ContextMutablePtr context, const String & create_
void CnchWorkerResource::executeCreateQuery(ContextMutablePtr context, const String & create_query, bool skip_if_exists, const ColumnsDescription & object_columns)
{
LOG_DEBUG(&Poco::Logger::get("WorkerResource"), "start create cloud table {}", create_query);
LOG_DEBUG(getLogger("WorkerResource"), "start create cloud table {}", create_query);
auto ast_query = parseCreateQuery(context, create_query);
auto & ast_create_query = ast_query->as<ASTCreateQuery &>();
@ -85,7 +85,7 @@ void CnchWorkerResource::executeCacheableCreateQuery(
const String & underlying_dictionary_tables,
const ColumnsDescription & object_columns)
{
static auto * log = &Poco::Logger::get("WorkerResource");
static auto log = getLogger("WorkerResource");
std::shared_ptr<StorageCloudMergeTree> cached;
if (auto cache = context->tryGetCloudTableDefinitionCache(); cache && !context->hasSessionTimeZone())
@ -220,7 +220,7 @@ void CnchWorkerResource::insertCloudTable(DatabaseAndTableName key, const Storag
}
}
static auto * log = &Poco::Logger::get("WorkerResource");
static auto log = getLogger("WorkerResource");
LOG_DEBUG(log, "Successfully create database {} and table {} {}",
tenant_db, storage->getName(), storage->getStorageID().getNameForLogs());
}

View File

@ -101,7 +101,7 @@ namespace ErrorCodes
CnchWorkerServiceImpl::CnchWorkerServiceImpl(ContextMutablePtr context_)
: WithMutableContext(context_->getGlobalContext())
, log(&Poco::Logger::get("CnchWorkerService"))
, log(getLogger("CnchWorkerService"))
, thread_pool(getNumberOfPhysicalCPUCores() * 4, getNumberOfPhysicalCPUCores() * 2, getNumberOfPhysicalCPUCores() * 8)
{
}

View File

@ -15,6 +15,7 @@
#pragma once
#include <Common/Logger.h>
#include <Interpreters/Context_fwd.h>
#include <Protos/cnch_worker_rpc.pb.h>
#include <Storages/MergeTree/MergeTreeDataPartCNCH.h>
@ -228,7 +229,7 @@ public:
google::protobuf::Closure * done) override;
private:
Poco::Logger * log;
LoggerPtr log;
// class PreloadHandler;
// std::shared_ptr<PreloadHandler> preload_handler;

Some files were not shown because too many files have changed in this diff Show More