mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #59564 from kitaisreal/revert-59509-revert-59375-poco-logger-small-refactoring
Revert "Revert "Poco Logger small refactoring""
This commit is contained in:
commit
cd4e304db0
@ -22,6 +22,7 @@
|
||||
#include <cstddef>
|
||||
#include <map>
|
||||
#include <vector>
|
||||
|
||||
#include "Poco/Channel.h"
|
||||
#include "Poco/Format.h"
|
||||
#include "Poco/Foundation.h"
|
||||
@ -871,21 +872,11 @@ public:
|
||||
/// If the Logger does not yet exist, it is created, based
|
||||
/// on its parent logger.
|
||||
|
||||
static LoggerPtr getShared(const std::string & name);
|
||||
static LoggerPtr getShared(const std::string & name, bool should_be_owned_by_shared_ptr_if_created = true);
|
||||
/// 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
|
||||
/// on its parent logger.
|
||||
///
|
||||
/// WARNING: This method is not thread safe. You should
|
||||
/// probably use get() instead.
|
||||
/// The only time this method should be used is during
|
||||
/// program initialization, when only one thread is running.
|
||||
|
||||
static Logger & create(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
|
||||
/// Creates and returns a reference to a Logger with the
|
||||
/// given name. The Logger's Channel and log level as set as
|
||||
@ -932,6 +923,16 @@ public:
|
||||
|
||||
static const std::string ROOT; /// The name of the root logger ("").
|
||||
|
||||
public:
|
||||
struct LoggerEntry
|
||||
{
|
||||
Poco::Logger * logger;
|
||||
bool owned_by_shared_ptr = false;
|
||||
};
|
||||
|
||||
using LoggerMap = std::unordered_map<std::string, LoggerEntry>;
|
||||
using LoggerMapIterator = LoggerMap::iterator;
|
||||
|
||||
protected:
|
||||
Logger(const std::string & name, Channel * pChannel, int level);
|
||||
~Logger();
|
||||
@ -940,12 +941,16 @@ 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);
|
||||
|
||||
private:
|
||||
static std::pair<Logger::LoggerMapIterator, bool> unsafeGet(const std::string & name, bool get_shared);
|
||||
static Logger * unsafeGetRawPtr(const std::string & name);
|
||||
static std::pair<LoggerMapIterator, bool> unsafeCreate(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
|
||||
static Logger & parent(const std::string & name);
|
||||
static std::pair<LoggerMapIterator, bool> add(Logger * pLogger);
|
||||
static std::optional<LoggerMapIterator> find(const std::string & name);
|
||||
static Logger * findRawPtr(const std::string & name);
|
||||
|
||||
Logger();
|
||||
Logger(const Logger &);
|
||||
Logger & operator=(const Logger &);
|
||||
|
@ -38,14 +38,7 @@ std::mutex & getLoggerMutex()
|
||||
return *logger_mutex;
|
||||
}
|
||||
|
||||
struct LoggerEntry
|
||||
{
|
||||
Poco::Logger * logger;
|
||||
bool owned_by_shared_ptr = false;
|
||||
};
|
||||
|
||||
using LoggerMap = std::unordered_map<std::string, LoggerEntry>;
|
||||
LoggerMap * _pLoggerMap = nullptr;
|
||||
Poco::Logger::LoggerMap * _pLoggerMap = nullptr;
|
||||
|
||||
}
|
||||
|
||||
@ -337,10 +330,12 @@ struct LoggerDeleter
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
inline LoggerPtr makeLoggerPtr(Logger & logger)
|
||||
inline LoggerPtr makeLoggerPtr(Logger & logger, bool owned_by_shared_ptr)
|
||||
{
|
||||
return std::shared_ptr<Logger>(&logger, LoggerDeleter());
|
||||
if (owned_by_shared_ptr)
|
||||
return LoggerPtr(&logger, LoggerDeleter());
|
||||
|
||||
return LoggerPtr(std::shared_ptr<void>{}, &logger);
|
||||
}
|
||||
|
||||
}
|
||||
@ -350,64 +345,67 @@ Logger& Logger::get(const std::string& name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
Logger & logger = unsafeGet(name);
|
||||
|
||||
/** If there are already shared pointer created for this logger
|
||||
* we need to increment Logger reference count and now logger
|
||||
* is owned by logger infrastructure.
|
||||
*/
|
||||
auto it = _pLoggerMap->find(name);
|
||||
if (it->second.owned_by_shared_ptr)
|
||||
{
|
||||
it->second.logger->duplicate();
|
||||
it->second.owned_by_shared_ptr = false;
|
||||
}
|
||||
|
||||
return logger;
|
||||
auto [it, inserted] = unsafeGet(name, false /*get_shared*/);
|
||||
return *it->second.logger;
|
||||
}
|
||||
|
||||
|
||||
LoggerPtr Logger::getShared(const std::string & name)
|
||||
LoggerPtr Logger::getShared(const std::string & name, bool should_be_owned_by_shared_ptr_if_created)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
bool logger_exists = _pLoggerMap && _pLoggerMap->contains(name);
|
||||
auto [it, inserted] = unsafeGet(name, true /*get_shared*/);
|
||||
|
||||
Logger & logger = unsafeGet(name);
|
||||
|
||||
/** If logger already exists, then this shared pointer does not own it.
|
||||
* If logger does not exists, logger infrastructure could be already destroyed
|
||||
* or logger was created.
|
||||
/** If during `unsafeGet` logger was created, then this shared pointer owns it.
|
||||
* If logger was already created, then this shared pointer does not own it.
|
||||
*/
|
||||
if (logger_exists)
|
||||
{
|
||||
logger.duplicate();
|
||||
}
|
||||
else if (_pLoggerMap)
|
||||
{
|
||||
_pLoggerMap->find(name)->second.owned_by_shared_ptr = true;
|
||||
}
|
||||
if (inserted && should_be_owned_by_shared_ptr_if_created)
|
||||
it->second.owned_by_shared_ptr = true;
|
||||
|
||||
return makeLoggerPtr(logger);
|
||||
return makeLoggerPtr(*it->second.logger, it->second.owned_by_shared_ptr);
|
||||
}
|
||||
|
||||
|
||||
Logger& Logger::unsafeGet(const std::string& name)
|
||||
std::pair<Logger::LoggerMapIterator, bool> Logger::unsafeGet(const std::string& name, bool get_shared)
|
||||
{
|
||||
Logger* pLogger = find(name);
|
||||
if (!pLogger)
|
||||
std::optional<Logger::LoggerMapIterator> optional_logger_it = find(name);
|
||||
|
||||
if (optional_logger_it)
|
||||
{
|
||||
auto & logger_it = *optional_logger_it;
|
||||
|
||||
if (logger_it->second.owned_by_shared_ptr)
|
||||
{
|
||||
logger_it->second.logger->duplicate();
|
||||
|
||||
if (!get_shared)
|
||||
logger_it->second.owned_by_shared_ptr = false;
|
||||
}
|
||||
}
|
||||
|
||||
if (!optional_logger_it)
|
||||
{
|
||||
Logger * logger = nullptr;
|
||||
|
||||
if (name == ROOT)
|
||||
{
|
||||
pLogger = new Logger(name, 0, Message::PRIO_INFORMATION);
|
||||
logger = new Logger(name, nullptr, Message::PRIO_INFORMATION);
|
||||
}
|
||||
else
|
||||
{
|
||||
Logger& par = parent(name);
|
||||
pLogger = new Logger(name, par.getChannel(), par.getLevel());
|
||||
logger = new Logger(name, par.getChannel(), par.getLevel());
|
||||
}
|
||||
add(pLogger);
|
||||
|
||||
return add(logger);
|
||||
}
|
||||
return *pLogger;
|
||||
|
||||
return std::make_pair(*optional_logger_it, false);
|
||||
}
|
||||
|
||||
|
||||
Logger * Logger::unsafeGetRawPtr(const std::string & name)
|
||||
{
|
||||
return unsafeGet(name, false /*get_shared*/).first->second.logger;
|
||||
}
|
||||
|
||||
|
||||
@ -415,24 +413,24 @@ Logger& Logger::create(const std::string& name, Channel* pChannel, int level)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return unsafeCreate(name, pChannel, level);
|
||||
return *unsafeCreate(name, pChannel, level).first->second.logger;
|
||||
}
|
||||
|
||||
LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int level)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
Logger & logger = unsafeCreate(name, pChannel, level);
|
||||
_pLoggerMap->find(name)->second.owned_by_shared_ptr = true;
|
||||
auto [it, inserted] = unsafeCreate(name, pChannel, level);
|
||||
it->second.owned_by_shared_ptr = true;
|
||||
|
||||
return makeLoggerPtr(logger);
|
||||
return makeLoggerPtr(*it->second.logger, it->second.owned_by_shared_ptr);
|
||||
}
|
||||
|
||||
Logger& Logger::root()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return unsafeGet(ROOT);
|
||||
return *unsafeGetRawPtr(ROOT);
|
||||
}
|
||||
|
||||
|
||||
@ -440,7 +438,11 @@ Logger* Logger::has(const std::string& name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return find(name);
|
||||
auto optional_it = find(name);
|
||||
if (!optional_it)
|
||||
return nullptr;
|
||||
|
||||
return (*optional_it)->second.logger;
|
||||
}
|
||||
|
||||
|
||||
@ -459,20 +461,32 @@ void Logger::shutdown()
|
||||
}
|
||||
|
||||
delete _pLoggerMap;
|
||||
_pLoggerMap = 0;
|
||||
_pLoggerMap = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Logger* Logger::find(const std::string& name)
|
||||
std::optional<Logger::LoggerMapIterator> Logger::find(const std::string& name)
|
||||
{
|
||||
if (_pLoggerMap)
|
||||
{
|
||||
LoggerMap::iterator it = _pLoggerMap->find(name);
|
||||
if (it != _pLoggerMap->end())
|
||||
return it->second.logger;
|
||||
return it;
|
||||
|
||||
return {};
|
||||
}
|
||||
return 0;
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
Logger * Logger::findRawPtr(const std::string & name)
|
||||
{
|
||||
auto optional_it = find(name);
|
||||
if (!optional_it)
|
||||
return nullptr;
|
||||
|
||||
return (*optional_it)->second.logger;
|
||||
}
|
||||
|
||||
|
||||
@ -490,28 +504,28 @@ void Logger::names(std::vector<std::string>& names)
|
||||
}
|
||||
}
|
||||
|
||||
Logger& Logger::unsafeCreate(const std::string & name, Channel * pChannel, int level)
|
||||
|
||||
std::pair<Logger::LoggerMapIterator, bool> 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;
|
||||
return add(pLogger);
|
||||
}
|
||||
|
||||
|
||||
Logger& Logger::parent(const std::string& name)
|
||||
{
|
||||
std::string::size_type pos = name.rfind('.');
|
||||
if (pos != std::string::npos)
|
||||
{
|
||||
std::string pname = name.substr(0, pos);
|
||||
Logger* pParent = find(pname);
|
||||
Logger* pParent = findRawPtr(pname);
|
||||
if (pParent)
|
||||
return *pParent;
|
||||
else
|
||||
return parent(pname);
|
||||
}
|
||||
else return unsafeGet(ROOT);
|
||||
else return *unsafeGetRawPtr(ROOT);
|
||||
}
|
||||
|
||||
|
||||
@ -579,12 +593,14 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
void Logger::add(Logger* pLogger)
|
||||
std::pair<Logger::LoggerMapIterator, bool> Logger::add(Logger* pLogger)
|
||||
{
|
||||
if (!_pLoggerMap)
|
||||
_pLoggerMap = new LoggerMap;
|
||||
_pLoggerMap = new Logger::LoggerMap;
|
||||
|
||||
_pLoggerMap->emplace(pLogger->name(), LoggerEntry{pLogger, false /*owned_by_shared_ptr*/});
|
||||
auto result = _pLoggerMap->emplace(pLogger->name(), LoggerEntry{pLogger, false /*owned_by_shared_ptr*/});
|
||||
assert(result.second);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <Poco/Channel.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Message.h>
|
||||
@ -24,6 +26,16 @@ using LoggerRawPtr = Poco::Logger *;
|
||||
*/
|
||||
LoggerPtr getLogger(const std::string & name);
|
||||
|
||||
/** Get Logger with specified name. If the Logger does not exists, it is created.
|
||||
* This overload was added for specific purpose, when logger is constructed from constexpr string.
|
||||
* Logger is destroyed only during program shutdown.
|
||||
*/
|
||||
template <size_t n>
|
||||
ALWAYS_INLINE LoggerPtr getLogger(const char (&name)[n])
|
||||
{
|
||||
return Poco::Logger::getShared(name, false /*should_be_owned_by_shared_ptr_if_created*/);
|
||||
}
|
||||
|
||||
/** Create Logger with specified name, channel and logging level.
|
||||
* If Logger already exists, throws exception.
|
||||
* Logger is destroyed, when last shared ptr that refers to Logger with specified name is destroyed.
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Poco/NullChannel.h>
|
||||
#include <Poco/StreamChannel.h>
|
||||
#include <sstream>
|
||||
#include <thread>
|
||||
|
||||
|
||||
TEST(Logger, Log)
|
||||
@ -100,3 +101,75 @@ TEST(Logger, SideEffects)
|
||||
|
||||
LOG_TRACE(log, "test no throw {}", getLogMessageParamOrThrow());
|
||||
}
|
||||
|
||||
TEST(Logger, SharedRawLogger)
|
||||
{
|
||||
{
|
||||
std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
auto stream_channel = Poco::AutoPtr<Poco::StreamChannel>(new Poco::StreamChannel(stream));
|
||||
|
||||
auto shared_logger = getLogger("Logger_1");
|
||||
shared_logger->setChannel(stream_channel.get());
|
||||
shared_logger->setLevel("trace");
|
||||
|
||||
LOG_TRACE(shared_logger, "SharedLogger1Log1");
|
||||
LOG_TRACE(getRawLogger("Logger_1"), "RawLogger1Log");
|
||||
LOG_TRACE(shared_logger, "SharedLogger1Log2");
|
||||
|
||||
auto actual = stream.str();
|
||||
EXPECT_EQ(actual, "SharedLogger1Log1\nRawLogger1Log\nSharedLogger1Log2\n");
|
||||
}
|
||||
{
|
||||
std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
auto stream_channel = Poco::AutoPtr<Poco::StreamChannel>(new Poco::StreamChannel(stream));
|
||||
|
||||
auto * raw_logger = getRawLogger("Logger_2");
|
||||
raw_logger->setChannel(stream_channel.get());
|
||||
raw_logger->setLevel("trace");
|
||||
|
||||
LOG_TRACE(getLogger("Logger_2"), "SharedLogger2Log1");
|
||||
LOG_TRACE(raw_logger, "RawLogger2Log");
|
||||
LOG_TRACE(getLogger("Logger_2"), "SharedLogger2Log2");
|
||||
|
||||
auto actual = stream.str();
|
||||
EXPECT_EQ(actual, "SharedLogger2Log1\nRawLogger2Log\nSharedLogger2Log2\n");
|
||||
}
|
||||
}
|
||||
|
||||
TEST(Logger, SharedLoggersThreadSafety)
|
||||
{
|
||||
static size_t threads_count = std::thread::hardware_concurrency();
|
||||
static constexpr size_t loggers_count = 10;
|
||||
static constexpr size_t logger_get_count = 1000;
|
||||
|
||||
Poco::Logger::root();
|
||||
|
||||
std::vector<std::string> names;
|
||||
|
||||
Poco::Logger::names(names);
|
||||
size_t loggers_size_before = names.size();
|
||||
|
||||
std::vector<std::thread> threads;
|
||||
|
||||
for (size_t thread_index = 0; thread_index < threads_count; ++thread_index)
|
||||
{
|
||||
threads.emplace_back([]()
|
||||
{
|
||||
for (size_t logger_index = 0; logger_index < loggers_count; ++logger_index)
|
||||
{
|
||||
for (size_t iteration = 0; iteration < logger_get_count; ++iteration)
|
||||
{
|
||||
getLogger("Logger_" + std::to_string(logger_index));
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
|
||||
Poco::Logger::names(names);
|
||||
size_t loggers_size_after = names.size();
|
||||
|
||||
EXPECT_EQ(loggers_size_before, loggers_size_after);
|
||||
}
|
||||
|
@ -102,7 +102,7 @@ void checkS3Capabilities(
|
||||
if (s3_capabilities.support_batch_delete && !checkBatchRemove(storage, key_with_trailing_slash))
|
||||
{
|
||||
LOG_WARNING(
|
||||
&Poco::Logger::get("S3ObjectStorage"),
|
||||
getLogger("S3ObjectStorage"),
|
||||
"Storage for disk {} does not support batch delete operations, "
|
||||
"so `s3_capabilities.support_batch_delete` was automatically turned off during the access check. "
|
||||
"To remove this message set `s3_capabilities.support_batch_delete` for the disk to `false`.",
|
||||
|
@ -82,7 +82,7 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lock<std::sha
|
||||
if (!inserted)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Loading data for {} more than once", file_path);
|
||||
|
||||
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Adding file: {}, size: {}", file_path, size);
|
||||
LOG_TRACE(getLogger("DiskWeb"), "Adding file: {}, size: {}", file_path, size);
|
||||
loaded_files.emplace_back(file_path);
|
||||
}
|
||||
|
||||
|
@ -76,7 +76,7 @@ namespace
|
||||
static std::once_flag once_flag;
|
||||
std::call_once(once_flag, [&config]
|
||||
{
|
||||
static LoggerPtr logger = getLogger("grpc");
|
||||
static LoggerRawPtr logger = getRawLogger("grpc");
|
||||
gpr_set_log_function([](gpr_log_func_args* args)
|
||||
{
|
||||
if (args->severity == GPR_LOG_SEVERITY_DEBUG)
|
||||
@ -622,7 +622,7 @@ namespace
|
||||
class Call
|
||||
{
|
||||
public:
|
||||
Call(CallType call_type_, std::unique_ptr<BaseResponder> responder_, IServer & iserver_, LoggerPtr log_);
|
||||
Call(CallType call_type_, std::unique_ptr<BaseResponder> responder_, IServer & iserver_, LoggerRawPtr log_);
|
||||
~Call();
|
||||
|
||||
void start(const std::function<void(void)> & on_finish_call_callback);
|
||||
@ -664,7 +664,7 @@ namespace
|
||||
const CallType call_type;
|
||||
std::unique_ptr<BaseResponder> responder;
|
||||
IServer & iserver;
|
||||
LoggerPtr log = nullptr;
|
||||
LoggerRawPtr log = nullptr;
|
||||
|
||||
std::optional<Session> session;
|
||||
ContextMutablePtr query_context;
|
||||
@ -726,7 +726,7 @@ namespace
|
||||
};
|
||||
// NOLINTEND(clang-analyzer-optin.performance.Padding)
|
||||
|
||||
Call::Call(CallType call_type_, std::unique_ptr<BaseResponder> responder_, IServer & iserver_, LoggerPtr log_)
|
||||
Call::Call(CallType call_type_, std::unique_ptr<BaseResponder> responder_, IServer & iserver_, LoggerRawPtr log_)
|
||||
: call_type(call_type_), responder(std::move(responder_)), iserver(iserver_), log(log_)
|
||||
{
|
||||
}
|
||||
@ -1851,7 +1851,7 @@ private:
|
||||
GRPCServer::GRPCServer(IServer & iserver_, const Poco::Net::SocketAddress & address_to_listen_)
|
||||
: iserver(iserver_)
|
||||
, address_to_listen(address_to_listen_)
|
||||
, log(getLogger("GRPCServer"))
|
||||
, log(getRawLogger("GRPCServer"))
|
||||
, runner(std::make_unique<Runner>(*this))
|
||||
{}
|
||||
|
||||
|
@ -3,10 +3,11 @@
|
||||
#include "config.h"
|
||||
|
||||
#if USE_GRPC
|
||||
|
||||
#include "clickhouse_grpc.grpc.pb.h"
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/Logger.h>
|
||||
#include "clickhouse_grpc.grpc.pb.h"
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
||||
@ -47,7 +48,7 @@ private:
|
||||
|
||||
IServer & iserver;
|
||||
const Poco::Net::SocketAddress address_to_listen;
|
||||
LoggerPtr log;
|
||||
LoggerRawPtr log;
|
||||
GRPCService grpc_service;
|
||||
std::unique_ptr<grpc::Server> grpc_server;
|
||||
std::unique_ptr<grpc::ServerCompletionQueue> queue;
|
||||
|
@ -59,7 +59,7 @@ int main(int argc, char *argv[])
|
||||
Poco::Logger::root().setChannel(channel);
|
||||
Poco::Logger::root().setLevel("trace");
|
||||
}
|
||||
auto * logger = &Poco::Logger::get("keeper-dumper");
|
||||
auto logger = getLogger("keeper-dumper");
|
||||
ResponsesQueue queue(std::numeric_limits<size_t>::max());
|
||||
SnapshotsQueue snapshots_queue{1};
|
||||
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
|
||||
|
Loading…
Reference in New Issue
Block a user