Merge branch 'ClickHouse:master' into #31363_format_template_configure_in_settings

This commit is contained in:
Shaun Struwig 2024-01-25 20:06:45 +01:00 committed by GitHub
commit e6844a5412
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
697 changed files with 1944 additions and 1392 deletions

View File

@ -33,7 +33,8 @@ namespace Poco
class Exception;
class Logger;
using LoggerPtr = std::shared_ptr<Logger>;
class Foundation_API Logger : public Channel
/// Logger is a special Channel that acts as the main
@ -870,6 +871,11 @@ public:
/// If the Logger does not yet exist, it is created, based
/// on its parent logger.
static LoggerPtr getShared(const std::string & name);
/// Returns a shared pointer to the Logger with the given name.
/// If the Logger does not yet exist, it is created, based
/// on its parent logger.
static Logger & unsafeGet(const std::string & name);
/// Returns a reference to the Logger with the given name.
/// If the Logger does not yet exist, it is created, based
@ -885,6 +891,11 @@ public:
/// given name. The Logger's Channel and log level as set as
/// specified.
static LoggerPtr createShared(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
/// Creates and returns a shared pointer to a Logger with the
/// given name. The Logger's Channel and log level as set as
/// specified.
static Logger & root();
/// Returns a reference to the root logger, which is the ultimate
/// ancestor of all Loggers.
@ -893,7 +904,7 @@ public:
/// Returns a pointer to the Logger with the given name if it
/// exists, or a null pointer otherwise.
static void destroy(const std::string & name);
static bool destroy(const std::string & name);
/// Destroys the logger with the specified name. Does nothing
/// if the logger is not found.
///
@ -938,6 +949,7 @@ protected:
void log(const std::string & text, Message::Priority prio, const char * file, int line);
static std::string format(const std::string & fmt, int argc, std::string argv[]);
static Logger & unsafeCreate(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
static Logger & parent(const std::string & name);
static void add(Logger * pLogger);
static Logger * find(const std::string & name);
@ -952,7 +964,6 @@ private:
std::atomic_int _level;
static LoggerMap * _pLoggerMap;
static Mutex _mapMtx;
};

View File

@ -38,15 +38,15 @@ public:
/// Creates the RefCountedObject.
/// The initial reference count is one.
void duplicate() const;
/// Increments the object's reference count.
size_t duplicate() const;
/// Increments the object's reference count, returns reference count before call.
void release() const throw();
size_t release() const throw();
/// Decrements the object's reference count
/// and deletes the object if the count
/// reaches zero.
/// reaches zero, returns reference count before call.
int referenceCount() const;
size_t referenceCount() const;
/// Returns the reference count.
protected:
@ -57,36 +57,40 @@ private:
RefCountedObject(const RefCountedObject &);
RefCountedObject & operator=(const RefCountedObject &);
mutable AtomicCounter _counter;
mutable std::atomic<size_t> _counter;
};
//
// inlines
//
inline int RefCountedObject::referenceCount() const
inline size_t RefCountedObject::referenceCount() const
{
return _counter.value();
return _counter.load(std::memory_order_acquire);
}
inline void RefCountedObject::duplicate() const
inline size_t RefCountedObject::duplicate() const
{
++_counter;
return _counter.fetch_add(1, std::memory_order_acq_rel);
}
inline void RefCountedObject::release() const throw()
inline size_t RefCountedObject::release() const throw()
{
size_t reference_count_before = _counter.fetch_sub(1, std::memory_order_acq_rel);
try
{
if (--_counter == 0)
if (reference_count_before == 1)
delete this;
}
catch (...)
{
poco_unexpected();
}
return reference_count_before;
}

View File

@ -20,12 +20,29 @@
#include "Poco/NumberParser.h"
#include "Poco/String.h"
#include <mutex>
namespace
{
std::mutex & getLoggerMutex()
{
auto get_logger_mutex_placeholder_memory = []()
{
static char buffer[sizeof(std::mutex)]{};
return buffer;
};
static std::mutex * logger_mutex = new (get_logger_mutex_placeholder_memory()) std::mutex();
return *logger_mutex;
}
}
namespace Poco {
Logger::LoggerMap* Logger::_pLoggerMap = 0;
Mutex Logger::_mapMtx;
const std::string Logger::ROOT;
@ -73,7 +90,7 @@ void Logger::setProperty(const std::string& name, const std::string& value)
setChannel(LoggingRegistry::defaultRegistry().channelForName(value));
else if (name == "level")
setLevel(value);
else
else
Channel::setProperty(name, value);
}
@ -112,14 +129,14 @@ void Logger::dump(const std::string& msg, const void* buffer, std::size_t length
void Logger::setLevel(const std::string& name, int level)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
std::string::size_type len = name.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
{
if (len == 0 ||
if (len == 0 ||
(it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.')))
{
it->second->setLevel(level);
@ -131,7 +148,7 @@ void Logger::setLevel(const std::string& name, int level)
void Logger::setChannel(const std::string& name, Channel* pChannel)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
@ -150,7 +167,7 @@ void Logger::setChannel(const std::string& name, Channel* pChannel)
void Logger::setProperty(const std::string& loggerName, const std::string& propertyName, const std::string& value)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
@ -280,13 +297,41 @@ void Logger::formatDump(std::string& message, const void* buffer, std::size_t le
}
namespace
{
struct LoggerDeleter
{
void operator()(Poco::Logger * logger)
{
if (Logger::destroy(logger->name()))
return;
logger->release();
}
};
inline LoggerPtr makeLoggerPtr(Logger & logger)
{
logger.duplicate();
return std::shared_ptr<Logger>(&logger, LoggerDeleter());
}
}
Logger& Logger::get(const std::string& name)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
return unsafeGet(name);
}
LoggerPtr Logger::getShared(const std::string & name)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
return makeLoggerPtr(unsafeGet(name));
}
Logger& Logger::unsafeGet(const std::string& name)
{
@ -310,18 +355,21 @@ Logger& Logger::unsafeGet(const std::string& name)
Logger& Logger::create(const std::string& name, Channel* pChannel, int level)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (find(name)) throw ExistsException();
Logger* pLogger = new Logger(name, pChannel, level);
add(pLogger);
return *pLogger;
return unsafeCreate(name, pChannel, level);
}
LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int level)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
return makeLoggerPtr(unsafeCreate(name, pChannel, level));
}
Logger& Logger::root()
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
return unsafeGet(ROOT);
}
@ -329,7 +377,7 @@ Logger& Logger::root()
Logger* Logger::has(const std::string& name)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
return find(name);
}
@ -337,7 +385,7 @@ Logger* Logger::has(const std::string& name)
void Logger::shutdown()
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
@ -363,25 +411,29 @@ Logger* Logger::find(const std::string& name)
}
void Logger::destroy(const std::string& name)
bool Logger::destroy(const std::string& name)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
LoggerMap::iterator it = _pLoggerMap->find(name);
if (it != _pLoggerMap->end())
{
it->second->release();
_pLoggerMap->erase(it);
if (it->second->release() == 1)
_pLoggerMap->erase(it);
return true;
}
}
return false;
}
void Logger::names(std::vector<std::string>& names)
{
Mutex::ScopedLock lock(_mapMtx);
std::lock_guard<std::mutex> lock(getLoggerMutex());
names.clear();
if (_pLoggerMap)
@ -393,6 +445,14 @@ void Logger::names(std::vector<std::string>& names)
}
}
Logger& Logger::unsafeCreate(const std::string & name, Channel * pChannel, int level)
{
if (find(name)) throw ExistsException();
Logger* pLogger = new Logger(name, pChannel, level);
add(pLogger);
return *pLogger;
}
Logger& Logger::parent(const std::string& name)
{

View File

@ -20,7 +20,7 @@ public:
const String & host_id_,
const String & proxy_database_name_,
ContextMutablePtr context_,
Poco::Logger * log_)
LoggerRawPtr log_)
: WithMutableContext(context_),
task_zookeeper_path(task_path_),
host_id(host_id_),
@ -230,7 +230,7 @@ private:
bool experimental_use_sample_offset{false};
Poco::Logger * log;
LoggerRawPtr log;
UInt64 max_table_tries = 3;
UInt64 max_shard_partition_tries = 3;

View File

@ -177,7 +177,7 @@ public:
auto watch_callback =
[my_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

@ -375,7 +375,7 @@ int KeeperClient::main(const std::vector<String> & /* args */)
if (!config().has("host") && !config().has("port") && !keys.empty())
{
LOG_INFO(&Poco::Logger::get("KeeperClient"), "Found keeper node in the config.xml, will use it for connection");
LOG_INFO(getLogger("KeeperClient"), "Found keeper node in the config.xml, will use it for connection");
for (const auto & key : keys)
{

View File

@ -28,7 +28,7 @@ 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");
LoggerPtr logger = getLogger("KeeperConverter");
logger->setChannel(console_channel);
if (options.count("help"))

View File

@ -624,7 +624,7 @@ catch (...)
void Keeper::logRevision() const
{
LOG_INFO(&Poco::Logger::get("Application"),
LOG_INFO(getLogger("Application"),
"Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}",
VERSION_STRING,
ClickHouseRevision::getVersionRevision(),

View File

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

View File

@ -31,7 +31,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

@ -9,40 +9,40 @@ const char DICT_LOGGER_NAME[] = "LibraryDictionarySourceExternal";
void ExternalDictionaryLibraryAPI::log(LogLevel level, CString msg)
{
auto & logger = Poco::Logger::get(DICT_LOGGER_NAME);
auto logger = getLogger(DICT_LOGGER_NAME);
switch (level)
{
case LogLevel::TRACE:
if (logger.trace())
logger.trace(msg);
if (logger->trace())
logger->trace(msg);
break;
case LogLevel::DEBUG:
if (logger.debug())
logger.debug(msg);
if (logger->debug())
logger->debug(msg);
break;
case LogLevel::INFORMATION:
if (logger.information())
logger.information(msg);
if (logger->information())
logger->information(msg);
break;
case LogLevel::NOTICE:
if (logger.notice())
logger.notice(msg);
if (logger->notice())
logger->notice(msg);
break;
case LogLevel::WARNING:
if (logger.warning())
logger.warning(msg);
if (logger->warning())
logger->warning(msg);
break;
case LogLevel::ERROR:
if (logger.error())
logger.error(msg);
if (logger->error())
logger->error(msg);
break;
case LogLevel::CRITICAL:
if (logger.critical())
logger.critical(msg);
if (logger->critical())
logger->critical(msg);
break;
case LogLevel::FATAL:
if (logger.fatal())
logger.fatal(msg);
if (logger->fatal())
logger->fatal(msg);
break;
}
}

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

@ -19,7 +19,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

@ -47,7 +47,7 @@ namespace
if (!response.sent())
*response.send() << message << '\n';
LOG_WARNING(&Poco::Logger::get("LibraryBridge"), fmt::runtime(message));
LOG_WARNING(getLogger("LibraryBridge"), fmt::runtime(message));
}
std::shared_ptr<Block> parseColumns(String && column_string)
@ -92,7 +92,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"))
{
}
@ -380,7 +380,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"))
{
}
@ -419,7 +419,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"))
{
}
@ -623,7 +623,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

@ -26,7 +26,7 @@ private:
static constexpr inline auto FORMAT = "RowBinary";
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
@ -40,7 +40,7 @@ public:
private:
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
@ -69,7 +69,7 @@ public:
private:
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
@ -83,7 +83,7 @@ public:
private:
const size_t keep_alive_timeout;
Poco::Logger * log;
LoggerPtr log;
};
}

View File

@ -221,7 +221,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();
LoggerRawPtr log = &logger();
std::filesystem::path parent_folder;
std::filesystem::path default_path;
@ -631,7 +631,7 @@ void LocalServer::processConfig()
tryInitPath();
Poco::Logger * log = &logger();
LoggerRawPtr log = &logger();
/// Maybe useless
if (config().has("macros"))

View File

@ -18,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_)
{
}
@ -26,7 +26,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
private:
Poco::Logger * log;
LoggerPtr log;
size_t keep_alive_timeout;
};

View File

@ -16,7 +16,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 +24,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
private:
Poco::Logger * log;
LoggerPtr log;
size_t keep_alive_timeout;
};

View File

@ -24,7 +24,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 +33,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
private:
Poco::Logger * log;
LoggerPtr log;
size_t keep_alive_timeout;
String mode;

View File

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

View File

@ -30,7 +30,7 @@ private:
column.insertFrom(sample_column, 0);
}
Poco::Logger * log;
LoggerPtr log;
const UInt64 max_block_size;
ExternalResultDescription description;

View File

@ -19,7 +19,7 @@ ODBCSink::ODBCSink(
ContextPtr local_context_,
IdentifierQuotingStyle quoting_)
: ISink(sample_block_)
, log(&Poco::Logger::get("ODBCSink"))
, log(getLogger("ODBCSink"))
, connection_holder(std::move(connection_holder_))
, db_name(remote_database_name_)
, table_name(remote_table_name_)

View File

@ -30,7 +30,7 @@ protected:
void consume(Chunk chunk) override;
private:
Poco::Logger * log;
LoggerPtr log;
nanodbc::ConnectionHolderPtr connection_holder;
std::string db_name;

View File

@ -11,7 +11,7 @@ namespace DB
ODBCBridgeHandlerFactory::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_)
{

View File

@ -22,7 +22,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

@ -97,7 +97,7 @@ T execute(nanodbc::ConnectionHolderPtr connection_holder, std::function<T(nanodb
/// https://docs.microsoft.com/ru-ru/sql/odbc/reference/appendixes/appendix-a-odbc-error-codes?view=sql-server-ver15
bool is_retriable = e.state().starts_with("08") || e.state().starts_with("24") || e.state().starts_with("25");
LOG_ERROR(
&Poco::Logger::get("ODBCConnection"),
getLogger("ODBCConnection"),
"ODBC query failed with error: {}, state: {}, native code: {}{}",
e.what(), e.state(), e.native(), is_retriable ? ", will retry" : "");

View File

@ -19,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_)
{
}
@ -27,7 +27,7 @@ public:
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) 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

@ -365,7 +365,7 @@ void Server::createServer(
namespace
{
void setOOMScore(int value, Poco::Logger * log)
void setOOMScore(int value, LoggerRawPtr log)
{
try
{
@ -450,7 +450,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;
@ -2491,7 +2491,7 @@ void Server::stopServers(
const ServerType & server_type
) const
{
Poco::Logger * log = &logger();
LoggerRawPtr log = &logger();
/// Remove servers once all their connections are closed
auto check_server = [&log](const char prefix[], auto & server)
@ -2530,7 +2530,7 @@ void Server::updateServers(
std::vector<ProtocolServerAdapter> & servers,
std::vector<ProtocolServerAdapter> & servers_to_start_before_tables)
{
Poco::Logger * log = &logger();
LoggerRawPtr log = &logger();
const auto listen_hosts = getListenHosts(config);
const auto interserver_listen_hosts = getInterserverListenHosts(config);

View File

@ -47,7 +47,7 @@ namespace
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;

View File

@ -443,7 +443,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={}",
title, level, node_name ? *node_name : "NULL", flags.toString(),
@ -1158,7 +1158,7 @@ AccessRights AccessRights::getFullAccess()
void AccessRights::logTree() const
{
auto * log = &Poco::Logger::get("AccessRights");
auto log = getLogger("AccessRights");
if (root)
{
root->logTree(log, "");

View File

@ -514,7 +514,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;
@ -556,7 +556,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

@ -298,7 +298,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;
if (params.use_default_roles)

View File

@ -185,9 +185,10 @@ private:
mutable std::atomic<bool> initialized = false; // can be removed after Bug 5504 is resolved
mutable std::atomic<bool> user_was_dropped = false;
mutable std::atomic<Poco::Logger *> trace_log = nullptr;
mutable std::mutex mutex;
/// TODO: Fix race
mutable LoggerPtr trace_log;
mutable UserPtr user TSA_GUARDED_BY(mutex);
mutable String user_name TSA_GUARDED_BY(mutex);
mutable scope_guard subscription_for_user_change TSA_GUARDED_BY(mutex);

View File

@ -47,7 +47,7 @@ namespace
}
AccessEntityPtr tryReadEntityFile(const String & file_path, Poco::Logger & log)
AccessEntityPtr tryReadEntityFile(const String & file_path, LoggerPtr log)
{
try
{
@ -55,7 +55,7 @@ namespace
}
catch (...)
{
tryLogCurrentException(&log);
tryLogCurrentException(log);
return nullptr;
}
}
@ -378,7 +378,7 @@ void DiskAccessStorage::reloadAllAndRebuildLists()
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

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

View File

@ -36,7 +36,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

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

View File

@ -3,6 +3,7 @@
#include "config.h"
#include <Access/Credentials.h>
#include <Common/Logger.h>
#include <base/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

@ -6,6 +6,7 @@
#include <Backups/BackupEntriesCollector.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <Common/callOnce.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Poco/UUIDGenerator.h>
@ -615,7 +616,7 @@ UUID IAccessStorage::generateRandomID()
}
void IAccessStorage::clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const Poco::Logger * log_)
void IAccessStorage::clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const LoggerPtr log_)
{
std::unordered_map<UUID, size_t> positions_by_id;
std::unordered_map<std::string_view, size_t> positions_by_type_and_name[static_cast<size_t>(AccessEntityType::MAX)];
@ -671,12 +672,13 @@ void IAccessStorage::clearConflictsInEntitiesList(std::vector<std::pair<UUID, Ac
}
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

@ -6,6 +6,7 @@
#include <Parsers/IParser.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Common/SettingsChanges.h>
#include <Common/callOnce.h>
#include <atomic>
#include <functional>
@ -225,9 +226,9 @@ protected:
SettingsChanges & settings) const;
virtual bool isAddressAllowed(const User & user, const Poco::Net::IPAddress & address) const;
static UUID generateRandomID();
Poco::Logger * getLogger() const;
LoggerPtr getLogger() const;
static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); }
static void clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const Poco::Logger * log_);
static void clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const LoggerPtr log_);
[[noreturn]] void throwNotFound(const UUID & id) const;
[[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const;
[[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type);
@ -246,7 +247,9 @@ protected:
private:
const String storage_name;
mutable std::atomic<Poco::Logger *> log = nullptr;
mutable OnceFlag log_initialized;
mutable LoggerPtr log = nullptr;
};

View File

@ -63,7 +63,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

@ -532,7 +532,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params)
for (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

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

View File

@ -37,7 +37,7 @@ SettingsAuthResponseParser::parse(const Poco::Net::HTTPResponse & response, std:
}
catch (...)
{
LOG_INFO(&Poco::Logger::get("HTTPAuthentication"), "Failed to parse settings from authentication response. Skip it.");
LOG_INFO(getLogger("HTTPAuthentication"), "Failed to parse settings from authentication response. Skip it.");
}
return result;
}

View File

@ -440,7 +440,7 @@ struct GroupArrayNodeGeneral : public GroupArrayNodeBase<GroupArrayNodeGeneral>
return node;
}
void insertInto(IColumn & column) { column.deserializeAndInsertFromArena(data()); }
void insertInto(IColumn & column) { std::ignore = column.deserializeAndInsertFromArena(data()); }
};
template <typename Node, bool has_sampler>

View File

@ -19,6 +19,7 @@
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/KeyHolderHelpers.h>
namespace DB
@ -153,8 +154,6 @@ private:
UInt64 threshold;
UInt64 reserved;
static void deserializeAndInsert(StringRef str, IColumn & data_to);
public:
AggregateFunctionTopKGeneric(
UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params)
@ -251,12 +250,7 @@ public:
offsets_to.push_back(offsets_to.back() + result_vec.size());
for (auto & elem : result_vec)
{
if constexpr (is_plain_column)
data_to.insertData(elem.key.data, elem.key.size);
else
data_to.deserializeAndInsertFromArena(elem.key.data);
}
deserializeAndInsert<is_plain_column>(elem.key, data_to);
}
};

View File

@ -29,7 +29,7 @@ static void deserializeAndInsert(StringRef str, IColumn & data_to)
if constexpr (is_plain_column)
data_to.insertData(str.data, str.size);
else
data_to.deserializeAndInsertFromArena(str.data);
std::ignore = data_to.deserializeAndInsertFromArena(str.data);
}
}

View File

@ -326,6 +326,68 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q
}
}
QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree)
{
QueryTreeNodes result;
QueryTreeNodes nodes_to_process;
nodes_to_process.push_back(tree);
while (!nodes_to_process.empty())
{
auto node_to_process = std::move(nodes_to_process.back());
nodes_to_process.pop_back();
auto node_type = node_to_process->getNodeType();
switch (node_type)
{
case QueryTreeNodeType::TABLE:
{
result.push_back(std::move(node_to_process));
break;
}
case QueryTreeNodeType::QUERY:
{
nodes_to_process.push_back(node_to_process->as<QueryNode>()->getJoinTree());
break;
}
case QueryTreeNodeType::UNION:
{
for (const auto & union_node : node_to_process->as<UnionNode>()->getQueries().getNodes())
nodes_to_process.push_back(union_node);
break;
}
case QueryTreeNodeType::TABLE_FUNCTION:
{
// Arguments of table function can't contain TableNodes.
break;
}
case QueryTreeNodeType::ARRAY_JOIN:
{
nodes_to_process.push_back(node_to_process->as<ArrayJoinNode>()->getTableExpression());
break;
}
case QueryTreeNodeType::JOIN:
{
auto & join_node = node_to_process->as<JoinNode &>();
nodes_to_process.push_back(join_node.getRightTableExpression());
nodes_to_process.push_back(join_node.getLeftTableExpression());
break;
}
default:
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected node type for table expression. "
"Expected table, table function, query, union, join or array join. Actual {}",
node_to_process->getNodeTypeName());
}
}
}
return result;
}
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join)
{
QueryTreeNodes result;

View File

@ -50,10 +50,13 @@ std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr &
*/
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options);
/// Extract table, table function, query, union from join tree
/// Extract all TableNodes from the query tree.
QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree);
/// Extract table, table function, query, union from join tree.
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false);
/// Extract left table expression from join tree
/// Extract left table expression from join tree.
QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node);
/** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree.

View File

@ -9,7 +9,7 @@ namespace DB
{
BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_)
: log(&Poco::Logger::get("BackupCoordinationLocal")), file_infos(plain_backup_)
: log(getLogger("BackupCoordinationLocal")), file_infos(plain_backup_)
{
}

View File

@ -57,7 +57,7 @@ public:
bool hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const override;
private:
Poco::Logger * const log;
LoggerPtr const log;
BackupCoordinationReplicatedTables TSA_GUARDED_BY(replicated_tables_mutex) replicated_tables;
BackupCoordinationReplicatedAccess TSA_GUARDED_BY(replicated_access_mutex) replicated_access;

View File

@ -173,7 +173,7 @@ BackupCoordinationRemote::BackupCoordinationRemote(
, current_host_index(findCurrentHostIndex(all_hosts, current_host))
, plain_backup(plain_backup_)
, is_internal(is_internal_)
, log(&Poco::Logger::get("BackupCoordinationRemote"))
, log(getLogger("BackupCoordinationRemote"))
, with_retries(
log,
get_zookeeper_,

View File

@ -102,7 +102,7 @@ private:
const size_t current_host_index;
const bool plain_backup;
const bool is_internal;
Poco::Logger * const log;
LoggerPtr const log;
/// The order of these two fields matters, because stage_sync holds a reference to with_retries object
mutable WithRetries with_retries;

View File

@ -24,7 +24,7 @@ namespace ErrorCodes
BackupCoordinationStageSync::BackupCoordinationStageSync(
const String & root_zookeeper_path_,
WithRetries & with_retries_,
Poco::Logger * log_)
LoggerPtr log_)
: zookeeper_path(root_zookeeper_path_ + "/stage")
, with_retries(with_retries_)
, log(log_)

View File

@ -12,7 +12,7 @@ public:
BackupCoordinationStageSync(
const String & root_zookeeper_path_,
WithRetries & with_retries_,
Poco::Logger * log_);
LoggerPtr log_);
/// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that.
void set(const String & current_host, const String & new_stage, const String & message, const bool & all_hosts = false);
@ -36,7 +36,7 @@ private:
String zookeeper_path;
/// A reference to the field of parent object - BackupCoordinationRemote or RestoreCoordinationRemote
WithRetries & with_retries;
Poco::Logger * log;
LoggerPtr log;
};
}

View File

@ -97,7 +97,7 @@ BackupEntriesCollector::BackupEntriesCollector(
, max_sleep_before_next_attempt_to_collect_metadata(
context->getConfigRef().getUInt64("backups.max_sleep_before_next_attempt_to_collect_metadata", 5000))
, compare_collected_metadata(context->getConfigRef().getBool("backups.compare_collected_metadata", true))
, log(&Poco::Logger::get("BackupEntriesCollector"))
, log(getLogger("BackupEntriesCollector"))
, global_zookeeper_retries_info(
context->getSettingsRef().backup_restore_keeper_max_retries,
context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms,

View File

@ -129,7 +129,7 @@ private:
/// Whether we should collect the metadata after a successful attempt one more time and check that nothing has changed.
const bool compare_collected_metadata;
Poco::Logger * log;
LoggerPtr log;
/// Unfortunately we can use ZooKeeper for collecting information for backup
/// and we need to retry...
ZooKeeperRetriesInfo global_zookeeper_retries_info;

View File

@ -102,7 +102,7 @@ BackupFileInfo buildFileInfoForBackupEntry(
const BackupEntryPtr & backup_entry,
const BackupPtr & base_backup,
const ReadSettings & read_settings,
Poco::Logger * log)
LoggerPtr log)
{
auto adjusted_path = removeLeadingSlash(file_name);
@ -129,7 +129,7 @@ BackupFileInfo buildFileInfoForBackupEntry(
}
if (!log)
log = &Poco::Logger::get("FileInfoFromBackupEntry");
log = getLogger("FileInfoFromBackupEntry");
std::optional<SizeAndChecksum> base_backup_file_info = getInfoAboutFileFromBaseBackupIfExists(base_backup, adjusted_path);
@ -216,7 +216,7 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr
std::exception_ptr exception;
auto thread_group = CurrentThread::getGroup();
Poco::Logger * log = &Poco::Logger::get("FileInfosFromBackupEntries");
LoggerPtr log = getLogger("FileInfosFromBackupEntries");
for (size_t i = 0; i != backup_entries.size(); ++i)
{

View File

@ -2,8 +2,7 @@
#include <Core/Types.h>
#include <Common/ThreadPool_fwd.h>
namespace Poco { class Logger; }
#include <Common/Logger.h>
namespace DB
{
@ -77,7 +76,7 @@ struct BackupFileInfo
using BackupFileInfos = std::vector<BackupFileInfo>;
/// Builds a BackupFileInfo for a specified backup entry.
BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, Poco::Logger * log);
BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, LoggerPtr log);
/// Builds a vector of BackupFileInfos for specified backup entries.
BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entries, const BackupPtr & base_backup, const ReadSettings & read_settings, ThreadPool & thread_pool, QueryStatusPtr process_list_element);

View File

@ -10,7 +10,7 @@
namespace DB
{
BackupReaderDefault::BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_)
BackupReaderDefault::BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_)
: log(log_)
, read_settings(read_settings_)
, write_settings(write_settings_)
@ -36,7 +36,7 @@ void BackupReaderDefault::copyFileToDisk(const String & path_in_backup, size_t f
write_buffer->finalize();
}
BackupWriterDefault::BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_)
BackupWriterDefault::BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_)
: log(log_)
, read_settings(read_settings_)
, write_settings(write_settings_)

View File

@ -18,7 +18,7 @@ enum class WriteMode;
class BackupReaderDefault : public IBackupReader
{
public:
BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_);
BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_);
~BackupReaderDefault() override = default;
/// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk.
@ -33,7 +33,7 @@ public:
size_t getWriteBufferSize() const override { return write_buffer_size; }
protected:
Poco::Logger * const log;
LoggerPtr const log;
const ReadSettings read_settings;
/// The write settings are used to write to the source disk in copyFileToDisk().
@ -45,7 +45,7 @@ protected:
class BackupWriterDefault : public IBackupWriter
{
public:
BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_);
BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_);
~BackupWriterDefault() override = default;
bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override;
@ -60,7 +60,7 @@ protected:
/// Here readFile() is used only to implement fileContentsEqual().
virtual std::unique_ptr<ReadBuffer> readFile(const String & file_name, size_t expected_file_size) = 0;
Poco::Logger * const log;
LoggerPtr const log;
/// The read settings are used to read from the source disk in copyFileFromDisk().
const ReadSettings read_settings;

View File

@ -9,7 +9,7 @@ namespace DB
{
BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
: BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderDisk"))
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderDisk"))
, disk(disk_)
, root_path(root_path_)
, data_source_description(disk->getDataSourceDescription())
@ -57,7 +57,7 @@ void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file
BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
: BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterDisk"))
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterDisk"))
, disk(disk_)
, root_path(root_path_)
, data_source_description(disk->getDataSourceDescription())

View File

@ -17,7 +17,7 @@ namespace ErrorCodes
}
BackupReaderFile::BackupReaderFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
: BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderFile"))
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderFile"))
, root_path(root_path_)
, data_source_description(DiskLocal::getLocalDataSourceDescription(root_path))
{
@ -75,7 +75,7 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file
BackupWriterFile::BackupWriterFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
: BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterFile"))
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterFile"))
, root_path(root_path_)
, data_source_description(DiskLocal::getLocalDataSourceDescription(root_path))
{

View File

@ -124,7 +124,7 @@ BackupReaderS3::BackupReaderS3(
const ReadSettings & read_settings_,
const WriteSettings & write_settings_,
const ContextPtr & context_)
: BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderS3"))
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3"))
, s3_uri(s3_uri_)
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false}
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()))
@ -214,7 +214,7 @@ BackupWriterS3::BackupWriterS3(
const ReadSettings & read_settings_,
const WriteSettings & write_settings_,
const ContextPtr & context_)
: BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterS3"))
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3"))
, s3_uri(s3_uri_)
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false}
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()))

View File

@ -105,7 +105,7 @@ BackupImpl::BackupImpl(
, version(INITIAL_BACKUP_VERSION)
, base_backup_info(base_backup_info_)
, use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_)
, log(&Poco::Logger::get("BackupImpl"))
, log(getLogger("BackupImpl"))
{
open();
}
@ -136,7 +136,7 @@ BackupImpl::BackupImpl(
, base_backup_info(base_backup_info_)
, deduplicate_files(deduplicate_files_)
, use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_)
, log(&Poco::Logger::get("BackupImpl"))
, log(getLogger("BackupImpl"))
{
open();
}

View File

@ -153,7 +153,7 @@ private:
bool writing_finalized = false;
bool deduplicate_files = true;
bool use_same_s3_credentials_for_base_backup = false;
const Poco::Logger * log;
const LoggerPtr log;
};
}

View File

@ -380,7 +380,7 @@ BackupsWorker::BackupsWorker(ContextMutablePtr global_context, size_t num_backup
, allow_concurrent_backups(allow_concurrent_backups_)
, allow_concurrent_restores(allow_concurrent_restores_)
, test_inject_sleep(test_inject_sleep_)
, log(&Poco::Logger::get("BackupsWorker"))
, log(getLogger("BackupsWorker"))
, backup_log(global_context->getBackupLog())
, process_list(global_context->getProcessList())
{

View File

@ -127,7 +127,7 @@ private:
const bool allow_concurrent_restores;
const bool test_inject_sleep;
Poco::Logger * log;
LoggerPtr log;
struct ExtendedOperationInfo
{

View File

@ -6,7 +6,7 @@
namespace DB
{
RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("RestoreCoordinationLocal"))
RestoreCoordinationLocal::RestoreCoordinationLocal() : log(getLogger("RestoreCoordinationLocal"))
{
}

View File

@ -51,7 +51,7 @@ public:
bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const override;
private:
Poco::Logger * const log;
LoggerPtr const log;
std::set<std::pair<String /* database_zk_path */, String /* table_name */>> acquired_tables_in_replicated_databases;
std::unordered_set<String /* table_zk_path */> acquired_data_in_replicated_tables;

View File

@ -32,7 +32,7 @@ RestoreCoordinationRemote::RestoreCoordinationRemote(
, current_host(current_host_)
, current_host_index(BackupCoordinationRemote::findCurrentHostIndex(all_hosts, current_host))
, is_internal(is_internal_)
, log(&Poco::Logger::get("RestoreCoordinationRemote"))
, log(getLogger("RestoreCoordinationRemote"))
, with_retries(
log,
get_zookeeper_,

View File

@ -73,7 +73,7 @@ private:
const String current_host;
const size_t current_host_index;
const bool is_internal;
Poco::Logger * const log;
LoggerPtr const log;
mutable WithRetries with_retries;
std::optional<BackupCoordinationStageSync> stage_sync;

View File

@ -89,7 +89,7 @@ RestorerFromBackup::RestorerFromBackup(
, process_list_element(context->getProcessListElement())
, on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000))
, create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000))
, log(&Poco::Logger::get("RestorerFromBackup"))
, log(getLogger("RestorerFromBackup"))
, tables_dependencies("RestorerFromBackup")
{
}

View File

@ -79,7 +79,7 @@ private:
QueryStatusPtr process_list_element;
std::chrono::milliseconds on_cluster_first_sync_timeout;
std::chrono::milliseconds create_table_timeout;
Poco::Logger * log;
LoggerPtr log;
Strings all_hosts;
DDLRenamingMap renaming_map;

View File

@ -21,7 +21,7 @@ WithRetries::KeeperSettings WithRetries::KeeperSettings::fromContext(ContextPtr
}
WithRetries::WithRetries(
Poco::Logger * log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_)
LoggerPtr log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_)
: log(log_)
, get_zookeeper(get_zookeeper_)
, settings(settings_)

View File

@ -52,7 +52,7 @@ public:
};
RetriesControlHolder createRetriesControlHolder(const String & name);
WithRetries(Poco::Logger * log, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback);
WithRetries(LoggerPtr log, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback);
/// Used to re-establish new connection inside a retry loop.
void renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const;
@ -62,7 +62,7 @@ private:
/// This will provide a special wrapper which is useful for testing
FaultyKeeper getFaultyZooKeeper() const;
Poco::Logger * log;
LoggerPtr log;
zkutil::GetZooKeeper get_zookeeper;
KeeperSettings settings;
QueryStatusPtr process_list_element;

View File

@ -51,7 +51,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

@ -31,7 +31,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 +40,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

@ -65,7 +65,7 @@ public:
const std::string & connection_string_,
bool use_connection_pooling_)
: IXDBCBridgeHelper(context_->getGlobalContext())
, log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
, log(getLogger(BridgeHelperMixin::getName() + "BridgeHelper"))
, connection_string(connection_string_)
, use_connection_pooling(use_connection_pooling_)
, http_timeout(http_timeout_)
@ -123,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(); }
@ -146,7 +146,7 @@ protected:
private:
using Configuration = Poco::Util::AbstractConfiguration;
Poco::Logger * log;
LoggerPtr log;
std::string connection_string;
bool use_connection_pooling;
Poco::Timespan http_timeout;

View File

@ -4,6 +4,7 @@
#include <Poco/Net/StreamSocket.h>
#include <Common/SSH/Wrappers.h>
#include <Common/callOnce.h>
#include <Client/IServerConnection.h>
#include <Core/Defines.h>
@ -244,16 +245,18 @@ 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;
OnceFlag log_initialized;
LoggerPtr log;
Connection & parent;
};

View File

@ -25,7 +25,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)
{
@ -114,7 +114,7 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync(
IConnectionPool * pool_,
const ConnectionTimeouts * timeouts_,
const Settings & settings_,
Poco::Logger * log_,
LoggerPtr log_,
const QualifiedTableName * table_to_check_)
: AsyncTaskExecutor(std::make_unique<Task>(*this)), connection_establisher(pool_, timeouts_, settings_, log_, table_to_check_)
{

View File

@ -23,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.
@ -38,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 +61,7 @@ public:
ConnectionEstablisherAsync(IConnectionPool * pool_,
const ConnectionTimeouts * timeouts_,
const Settings & settings_,
Poco::Logger * log_,
LoggerPtr log_,
const QualifiedTableName * table_to_check_ = nullptr);
/// Get file descriptor that can be added in epoll and be polled,

View File

@ -64,7 +64,7 @@ public:
Protocol::Secure secure_,
Priority priority_ = Priority{1})
: 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"))
, get_priority_load_balancing(load_balancing)
{
const std::string & local_hostname = getFQDNOrHostName();

View File

@ -34,7 +34,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory(
: pool(pool_)
, timeouts(timeouts_)
, table_to_check(table_to_check_)
, log(&Poco::Logger::get("HedgedConnectionsFactory"))
, log(getLogger("HedgedConnectionsFactory"))
, max_tries(max_tries_)
, fallback_to_stale_replicas(fallback_to_stale_replicas_)
, max_parallel_replicas(max_parallel_replicas_)

View File

@ -133,7 +133,7 @@ private:
std::shared_ptr<QualifiedTableName> table_to_check;
int last_used_index = -1;
Epoll epoll;
Poco::Logger * log;
LoggerPtr log;
std::string fail_messages;
/// The maximum number of attempts to connect to replicas.

View File

@ -222,11 +222,11 @@ public:
/// Deserializes a value that was serialized using IColumn::serializeValueIntoArena method.
/// Returns pointer to the position after the read data.
virtual const char * deserializeAndInsertFromArena(const char * pos) = 0;
[[nodiscard]] virtual const char * deserializeAndInsertFromArena(const char * pos) = 0;
/// Skip previously serialized value that was serialized using IColumn::serializeValueIntoArena method.
/// Returns a pointer to the position after the deserialized data.
virtual const char * skipSerializedInArena(const char *) const = 0;
[[nodiscard]] virtual const char * skipSerializedInArena(const char *) const = 0;
/// Update state of hash function with value of n-th element.
/// On subsequent calls of this method for sequence of column values of arbitrary types,

View File

@ -50,7 +50,7 @@ void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_)
auto [buf, len] = adjustToPageSize(buf_, len_, page_size);
if (::madvise(buf, len, MADV_POPULATE_WRITE) < 0)
LOG_TRACE(
LogFrequencyLimiter(&Poco::Logger::get("Allocator"), 1),
LogFrequencyLimiter(getLogger("Allocator"), 1),
"Attempt to populate pages failed: {} (EINVAL is expected for kernels < 5.14)",
errnoToString(errno));
#endif

View File

@ -34,7 +34,7 @@ namespace ErrorCodes
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch)
void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopwatch & watch)
{
if (total && (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)))
{
@ -205,7 +205,7 @@ void LoadTask::detach()
AsyncLoader::AsyncLoader(std::vector<PoolInitializer> pool_initializers, bool log_failures_, bool log_progress_)
: log_failures(log_failures_)
, log_progress(log_progress_)
, log(&Poco::Logger::get("AsyncLoader"))
, log(getLogger("AsyncLoader"))
{
pools.reserve(pool_initializers.size());
for (auto && init : pool_initializers)

View File

@ -15,6 +15,7 @@
#include <Common/Priority.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool_fwd.h>
#include <Common/Logger.h>
namespace Poco { class Logger; }
@ -40,7 +41,7 @@ using LoadTaskPtr = std::shared_ptr<LoadTask>;
using LoadTaskPtrs = std::vector<LoadTaskPtr>;
class AsyncLoader;
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch);
void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopwatch & watch);
// Execution status of a load job.
enum class LoadStatus
@ -419,7 +420,7 @@ private:
// Logging
const bool log_failures; // Worker should log all exceptions caught from job functions.
const bool log_progress; // Periodically log total progress
Poco::Logger * log;
LoggerPtr log;
mutable std::mutex mutex; // Guards all the fields below.
bool is_running = true;

View File

@ -58,7 +58,7 @@ AsynchronousMetrics::AsynchronousMetrics(
int update_period_seconds,
const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: update_period(update_period_seconds)
, log(&Poco::Logger::get("AsynchronousMetrics"))
, log(getLogger("AsynchronousMetrics"))
, protocol_server_metrics_func(protocol_server_metrics_func_)
{
#if defined(OS_LINUX)
@ -125,7 +125,7 @@ void AsynchronousMetrics::openSensors() TSA_REQUIRES(data_mutex)
catch (const ErrnoException & e)
{
LOG_WARNING(
&Poco::Logger::get("AsynchronousMetrics"),
getLogger("AsynchronousMetrics"),
"Thermal monitor '{}' exists but could not be read: {}.",
thermal_device_index,
errnoToString(e.getErrno()));
@ -254,7 +254,7 @@ void AsynchronousMetrics::openSensorsChips() TSA_REQUIRES(data_mutex)
catch (const ErrnoException & e)
{
LOG_WARNING(
&Poco::Logger::get("AsynchronousMetrics"),
getLogger("AsynchronousMetrics"),
"Hardware monitor '{}', sensor '{}' exists but could not be read: {}.",
hwmon_name,
sensor_index,

View File

@ -82,7 +82,7 @@ public:
protected:
const Duration update_period;
Poco::Logger * log;
LoggerPtr log;
private:
virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) = 0;
virtual void logImpl(AsynchronousMetricValues &) {}

View File

@ -77,23 +77,17 @@ ConfigProcessor::ConfigProcessor(
, name_pool(new Poco::XML::NamePool(65521))
, dom_parser(name_pool)
{
if (log_to_console && !Poco::Logger::has("ConfigProcessor"))
if (log_to_console && !hasLogger("ConfigProcessor"))
{
channel_ptr = new Poco::ConsoleChannel;
log = &Poco::Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE);
log = createLogger("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE);
}
else
{
log = &Poco::Logger::get("ConfigProcessor");
log = getLogger("ConfigProcessor");
}
}
ConfigProcessor::~ConfigProcessor()
{
if (channel_ptr) /// This means we have created a new console logger in the constructor.
Poco::Logger::destroy("ConfigProcessor");
}
static std::unordered_map<std::string, std::string_view> embedded_configs;
void ConfigProcessor::registerEmbeddedConfig(std::string name, std::string_view content)

View File

@ -7,6 +7,8 @@
#include <vector>
#include <memory>
#include <Common/Logger.h>
#include <Poco/DOM/Document.h>
#include <Poco/DOM/DOMParser.h>
#include <Poco/DOM/DOMWriter.h>
@ -44,8 +46,6 @@ public:
bool log_to_console = false,
const Substitutions & substitutions = Substitutions());
~ConfigProcessor();
/// Perform config includes and substitutions and return the resulting XML-document.
///
/// Suppose path is "/path/file.xml"
@ -125,7 +125,7 @@ private:
bool throw_on_bad_incl;
Poco::Logger * log;
LoggerPtr log;
Poco::AutoPtr<Poco::Channel> channel_ptr;
Substitutions substitutions;

View File

@ -69,7 +69,7 @@ private:
static constexpr auto reload_interval = std::chrono::seconds(2);
Poco::Logger * log = &Poco::Logger::get("ConfigReloader");
LoggerPtr log = getLogger("ConfigReloader");
std::string config_path;
std::vector<std::string> extra_paths;

View File

@ -104,7 +104,7 @@ DNSResolver::IPAddresses hostByName(const std::string & host)
}
catch (const Poco::Net::DNSException & e)
{
LOG_WARNING(&Poco::Logger::get("DNSResolver"), "Cannot resolve host ({}), error {}: {}.", host, e.code(), e.name());
LOG_WARNING(getLogger("DNSResolver"), "Cannot resolve host ({}), error {}: {}.", host, e.code(), e.name());
addresses.clear();
}
@ -201,7 +201,7 @@ struct DNSResolver::Impl
};
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), log(&Poco::Logger::get("DNSResolver")) {}
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), log(getLogger("DNSResolver")) {}
Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host)
{

View File

@ -73,7 +73,7 @@ private:
struct Impl;
std::unique_ptr<Impl> impl;
Poco::Logger * log;
LoggerPtr log;
/// Updates cached value and returns true it has been changed.
bool updateHost(const String & host);

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