Revert "Revert "Poco Logger small refactoring""

This commit is contained in:
Maksim Kita 2024-02-05 12:33:17 +03:00
parent 69e118e587
commit dc52def9eb
9 changed files with 255 additions and 113 deletions

View File

@ -166,6 +166,12 @@ set (SRCS
)
add_library (_poco_foundation ${SRCS})
target_link_libraries (_poco_foundation
PUBLIC
boost::headers_only
boost::system
)
add_library (Poco::Foundation ALIAS _poco_foundation)
# TODO: remove these warning exclusions

View File

@ -22,6 +22,9 @@
#include <cstddef>
#include <map>
#include <vector>
#include <boost/smart_ptr/intrusive_ptr.hpp>
#include "Poco/Channel.h"
#include "Poco/Format.h"
#include "Poco/Foundation.h"
@ -34,7 +37,7 @@ namespace Poco
class Exception;
class Logger;
using LoggerPtr = std::shared_ptr<Logger>;
using LoggerPtr = boost::intrusive_ptr<Logger>;
class Foundation_API Logger : public Channel
/// Logger is a special Channel that acts as the main
@ -871,21 +874,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 +925,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 +943,19 @@ 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);
friend void intrusive_ptr_add_ref(Logger * ptr);
friend void intrusive_ptr_release(Logger * ptr);
Logger();
Logger(const Logger &);
Logger & operator=(const Logger &);

View File

@ -53,11 +53,10 @@ protected:
virtual ~RefCountedObject();
/// Destroys the RefCountedObject.
mutable std::atomic<size_t> _counter;
private:
RefCountedObject(const RefCountedObject &);
RefCountedObject & operator=(const RefCountedObject &);
mutable std::atomic<size_t> _counter;
};

View File

@ -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;
}
@ -309,38 +302,9 @@ void Logger::formatDump(std::string& message, const void* buffer, std::size_t le
namespace
{
struct LoggerDeleter
{
void operator()(Poco::Logger * logger)
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
/// If logger infrastructure is destroyed just decrement logger reference count
if (!_pLoggerMap)
{
logger->release();
return;
}
auto it = _pLoggerMap->find(logger->name());
assert(it != _pLoggerMap->end());
/** If reference count is 1, this means this shared pointer owns logger
* and need destroy it.
*/
size_t reference_count_before_release = logger->release();
if (reference_count_before_release == 1)
{
assert(it->second.owned_by_shared_ptr);
_pLoggerMap->erase(it);
}
}
};
inline LoggerPtr makeLoggerPtr(Logger & logger)
{
return std::shared_ptr<Logger>(&logger, LoggerDeleter());
return LoggerPtr(&logger, false /*add_ref*/);
}
}
@ -350,64 +314,87 @@ 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)
if (inserted)
{
logger.duplicate();
}
else if (_pLoggerMap)
{
_pLoggerMap->find(name)->second.owned_by_shared_ptr = true;
if (should_be_owned_by_shared_ptr_if_created)
it->second.owned_by_shared_ptr = true;
else
it->second.logger->duplicate();
}
return makeLoggerPtr(logger);
return makeLoggerPtr(*it->second.logger);
}
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);
bool should_recreate_logger = false;
if (optional_logger_it)
{
auto & logger_it = *optional_logger_it;
std::optional<size_t> reference_count_before;
if (get_shared)
{
reference_count_before = logger_it->second.logger->duplicate();
}
else if (logger_it->second.owned_by_shared_ptr)
{
reference_count_before = logger_it->second.logger->duplicate();
logger_it->second.owned_by_shared_ptr = false;
}
/// Other thread already decided to delete this logger, but did not yet remove it from map
if (reference_count_before && reference_count_before == 0)
should_recreate_logger = true;
}
if (!optional_logger_it || should_recreate_logger)
{
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);
if (should_recreate_logger)
{
(*optional_logger_it)->second.logger = logger;
return std::make_pair(*optional_logger_it, true);
}
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 +402,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);
}
Logger& Logger::root()
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
return unsafeGet(ROOT);
return *unsafeGetRawPtr(ROOT);
}
@ -440,7 +427,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 +450,69 @@ 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;
}
void intrusive_ptr_add_ref(Logger * ptr)
{
ptr->duplicate();
}
void intrusive_ptr_release(Logger * ptr)
{
size_t reference_count_before = ptr->_counter.fetch_sub(1, std::memory_order_acq_rel);
if (reference_count_before != 1)
return;
{
std::lock_guard<std::mutex> lock(getLoggerMutex());
if (_pLoggerMap)
{
auto it = _pLoggerMap->find(ptr->name());
/** It is possible that during release other thread created logger and
* updated iterator in map.
*/
if (it != _pLoggerMap->end() && ptr == it->second.logger)
{
/** If reference count is 0, this means this intrusive pointer owns logger
* and need destroy it.
*/
assert(it->second.owned_by_shared_ptr);
_pLoggerMap->erase(it);
}
}
}
delete ptr;
}
@ -490,28 +530,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 +619,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;
}

View File

@ -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.

View File

@ -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);
}

View File

@ -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`.",

View File

@ -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);
}

View File

@ -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>();