mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 12:14:18 +00:00
Merge pull request #69657 from petern48/regexp_logger
Filter logging using regular expressions
This commit is contained in:
commit
f2ca21134f
@ -952,6 +952,8 @@ private:
|
||||
static std::pair<LoggerMapIterator, bool> add(Logger * pLogger);
|
||||
static std::optional<LoggerMapIterator> find(const std::string & name);
|
||||
static Logger * findRawPtr(const std::string & name);
|
||||
void unsafeSetChannel(Channel * pChannel);
|
||||
Channel* unsafeGetChannel() const;
|
||||
|
||||
Logger();
|
||||
Logger(const Logger &);
|
||||
|
@ -61,6 +61,13 @@ Logger::~Logger()
|
||||
|
||||
|
||||
void Logger::setChannel(Channel* pChannel)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
unsafeSetChannel(pChannel);
|
||||
}
|
||||
|
||||
|
||||
void Logger::unsafeSetChannel(Channel* pChannel)
|
||||
{
|
||||
if (_pChannel) _pChannel->release();
|
||||
_pChannel = pChannel;
|
||||
@ -69,6 +76,14 @@ void Logger::setChannel(Channel* pChannel)
|
||||
|
||||
|
||||
Channel* Logger::getChannel() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return unsafeGetChannel();
|
||||
}
|
||||
|
||||
|
||||
Channel* Logger::unsafeGetChannel() const
|
||||
{
|
||||
return _pChannel;
|
||||
}
|
||||
@ -89,7 +104,7 @@ void Logger::setLevel(const std::string& level)
|
||||
void Logger::setProperty(const std::string& name, const std::string& value)
|
||||
{
|
||||
if (name == "channel")
|
||||
setChannel(LoggingRegistry::defaultRegistry().channelForName(value));
|
||||
unsafeSetChannel(LoggingRegistry::defaultRegistry().channelForName(value));
|
||||
else if (name == "level")
|
||||
setLevel(value);
|
||||
else
|
||||
@ -160,7 +175,7 @@ void Logger::setChannel(const std::string& name, Channel* pChannel)
|
||||
if (len == 0 ||
|
||||
(it.first.compare(0, len, name) == 0 && (it.first.length() == len || it.first[len] == '.')))
|
||||
{
|
||||
it.second.logger->setChannel(pChannel);
|
||||
it.second.logger->unsafeSetChannel(pChannel);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -393,7 +408,7 @@ std::pair<Logger::LoggerMapIterator, bool> Logger::unsafeGet(const std::string&
|
||||
else
|
||||
{
|
||||
Logger& par = parent(name);
|
||||
logger = new Logger(name, par.getChannel(), par.getLevel());
|
||||
logger = new Logger(name, par.unsafeGetChannel(), par.getLevel());
|
||||
}
|
||||
|
||||
return add(logger);
|
||||
|
@ -1488,6 +1488,8 @@ Keys:
|
||||
- `formatting` – Log format for console output. Currently, only `json` is supported).
|
||||
- `use_syslog` - Also forward log output to syslog.
|
||||
- `syslog_level` - Log level for logging to syslog.
|
||||
- `message_regexp` - Only log messages that match this regular expression. Defaults to `""`, indicating no filtering.
|
||||
- `message_regexp_negative` - Only log messages that don't match this regular expression. Defaults to `""`, indicating no filtering.
|
||||
|
||||
**Log format specifiers**
|
||||
|
||||
@ -1576,6 +1578,28 @@ The log level of individual log names can be overridden. For example, to mute al
|
||||
</logger>
|
||||
```
|
||||
|
||||
**Regular Expression Filtering**
|
||||
|
||||
The messages logged can be filtered using regular expressions using `message_regexp` and `message_regexp_negative`. This can be done on a per-level basis or globally. If both a global and logger-specific pattern is specified, the global pattern is overridden (ignored) and only the logger-specific pattern applies. The positive and negative patterns are considered independently for this situation. Note: Using this feature may cause a slight slowdown in performance.
|
||||
|
||||
|
||||
```xml
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<!-- Global: Don't log Trace messages -->
|
||||
<message_regexp_negative>.*Trace.*</message_regexp_negative>
|
||||
|
||||
<message_regexps>
|
||||
<logger>
|
||||
<!-- For the executeQuery logger, only log if message has "Read", but not "from" -->
|
||||
<name>executeQuery</name>
|
||||
<message_regexp>.*Read.*</message_regexp>
|
||||
<message_regexp_negative>.*from.*</message_regexp_negative>
|
||||
</logger>
|
||||
</message_regexps>
|
||||
</logger>
|
||||
```
|
||||
|
||||
### syslog
|
||||
|
||||
To write log messages additionally to syslog:
|
||||
|
@ -1,12 +1,14 @@
|
||||
#include "Loggers.h"
|
||||
|
||||
#include "OwnFormattingChannel.h"
|
||||
#include "OwnPatternFormatter.h"
|
||||
#include "OwnSplitChannel.h"
|
||||
#include <Loggers/OwnFilteringChannel.h>
|
||||
#include <Loggers/OwnFormattingChannel.h>
|
||||
#include <Loggers/OwnPatternFormatter.h>
|
||||
#include <Loggers/OwnSplitChannel.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <sstream>
|
||||
|
||||
#include <Poco/AutoPtr.h>
|
||||
#include <Poco/ConsoleChannel.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Net/RemoteSyslogChannel.h>
|
||||
@ -222,6 +224,18 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
logger.close();
|
||||
|
||||
logger.setChannel(split);
|
||||
|
||||
const std::string global_pos_pattern = config.getRawString("logger.message_regexp", "");
|
||||
const std::string global_neg_pattern = config.getRawString("logger.message_regexp_negative", "");
|
||||
|
||||
Poco::AutoPtr<OwnPatternFormatter> pf;
|
||||
if (config.getString("logger.formatting.type", "") == "json")
|
||||
pf = new OwnJSONPatternFormatter(config);
|
||||
else
|
||||
pf = new OwnPatternFormatter;
|
||||
|
||||
DB::createOrUpdateFilterChannel(logger, global_pos_pattern, global_neg_pattern, pf, Poco::Logger::ROOT);
|
||||
|
||||
logger.setLevel(max_log_level);
|
||||
|
||||
// Global logging level and channel (it can be overridden for specific loggers).
|
||||
@ -236,6 +250,8 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
{
|
||||
logger.get(name).setLevel(max_log_level);
|
||||
logger.get(name).setChannel(split);
|
||||
|
||||
DB::createOrUpdateFilterChannel(logger.get(name), global_pos_pattern, global_neg_pattern, pf, name);
|
||||
}
|
||||
|
||||
// Explicitly specified log levels for specific loggers.
|
||||
@ -262,6 +278,26 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
}
|
||||
}
|
||||
}
|
||||
// Explicitly specified regexp patterns for filtering specific loggers
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys loggers_regexp;
|
||||
config.keys("logger.message_regexps", loggers_regexp);
|
||||
|
||||
if (!loggers_regexp.empty())
|
||||
{
|
||||
for (const auto & key : loggers_regexp)
|
||||
{
|
||||
if (key == "logger" || key.starts_with("logger["))
|
||||
{
|
||||
const std::string name = config.getString("logger.message_regexps." + key + ".name");
|
||||
const std::string pos_pattern = config.getRawString("logger.message_regexps." + key + ".message_regexp", global_pos_pattern);
|
||||
const std::string neg_pattern = config.getRawString("logger.message_regexps." + key + ".message_regexp_negative", global_neg_pattern);
|
||||
|
||||
DB::createOrUpdateFilterChannel(logger.root().get(name), pos_pattern, neg_pattern, pf, name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
if (allowTextLog() && config.has("text_log"))
|
||||
{
|
||||
@ -347,16 +383,32 @@ void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
}
|
||||
split->setLevel("syslog", syslog_level);
|
||||
|
||||
const std::string global_pos_pattern = config.getRawString("logger.message_regexp", "");
|
||||
const std::string global_neg_pattern = config.getRawString("logger.message_regexp_negative", "");
|
||||
|
||||
Poco::AutoPtr<OwnPatternFormatter> pf;
|
||||
if (config.getString("logger.formatting.type", "") == "json")
|
||||
pf = new OwnJSONPatternFormatter(config);
|
||||
else
|
||||
pf = new OwnPatternFormatter;
|
||||
|
||||
DB::createOrUpdateFilterChannel(logger, global_pos_pattern, global_neg_pattern, pf, Poco::Logger::ROOT);
|
||||
|
||||
// Global logging level (it can be overridden for specific loggers).
|
||||
logger.setLevel(max_log_level);
|
||||
|
||||
// Set level to all already created loggers
|
||||
std::vector<std::string> names;
|
||||
|
||||
logger.root().names(names);
|
||||
|
||||
// Set all to global in case logger.levels are not specified
|
||||
for (const auto & name : names)
|
||||
{
|
||||
logger.root().get(name).setLevel(max_log_level);
|
||||
|
||||
DB::createOrUpdateFilterChannel(logger.root().get(name), global_pos_pattern, global_neg_pattern, pf, name);
|
||||
}
|
||||
|
||||
logger.root().setLevel(max_log_level);
|
||||
|
||||
// Explicitly specified log levels for specific loggers.
|
||||
@ -383,6 +435,27 @@ void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Explicitly specified regexp patterns for filtering specific loggers
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys loggers_regexp;
|
||||
config.keys("logger.message_regexps", loggers_regexp);
|
||||
|
||||
if (!loggers_regexp.empty())
|
||||
{
|
||||
for (const auto & key : loggers_regexp)
|
||||
{
|
||||
if (key == "logger" || key.starts_with("logger["))
|
||||
{
|
||||
const std::string name(config.getString("logger.message_regexps." + key + ".name"));
|
||||
const std::string pos_pattern(config.getRawString("logger.message_regexps." + key + ".message_regexp", global_pos_pattern));
|
||||
const std::string neg_pattern(config.getRawString("logger.message_regexps." + key + ".message_regexp_negative", global_neg_pattern));
|
||||
|
||||
DB::createOrUpdateFilterChannel(logger.root().get(name), pos_pattern, neg_pattern, pf, name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// NOLINTEND(readability-static-accessed-through-instance)
|
||||
|
96
src/Loggers/OwnFilteringChannel.cpp
Normal file
96
src/Loggers/OwnFilteringChannel.cpp
Normal file
@ -0,0 +1,96 @@
|
||||
#include <shared_mutex>
|
||||
#include <Loggers/OwnFilteringChannel.h>
|
||||
#include <Poco/RegularExpression.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void OwnFilteringChannel::log(const Poco::Message & msg)
|
||||
{
|
||||
if (regexpFilteredOut(msg))
|
||||
return;
|
||||
|
||||
pChannel->log(msg);
|
||||
}
|
||||
|
||||
bool OwnFilteringChannel::regexpFilteredOut(const Poco::Message & msg)
|
||||
{
|
||||
std::string formatted_text;
|
||||
auto [pos_pattern, neg_pattern] = safeGetPatterns();
|
||||
|
||||
// Skip checks if both patterns are empty
|
||||
if (!pos_pattern.empty() || !neg_pattern.empty())
|
||||
{
|
||||
// Apply formatting to the text
|
||||
if (pFormatter)
|
||||
{
|
||||
pFormatter->formatExtended(ExtendedLogMessage::getFrom(msg), formatted_text);
|
||||
}
|
||||
else
|
||||
{
|
||||
formatted_text = msg.getText();
|
||||
}
|
||||
|
||||
// Check for patterns in formatted text
|
||||
Poco::RegularExpression positive_regexp(pos_pattern);
|
||||
if (!pos_pattern.empty() && !positive_regexp.match(formatted_text))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
Poco::RegularExpression negative_regexp(neg_pattern);
|
||||
if (!neg_pattern.empty() && negative_regexp.match(formatted_text))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void OwnFilteringChannel::setRegexpPatterns(const std::string & new_pos_pattern, const std::string & new_neg_pattern)
|
||||
{
|
||||
auto [old_pos_pattern, old_neg_pattern] = safeGetPatterns();
|
||||
if (old_pos_pattern != new_pos_pattern || old_neg_pattern != new_neg_pattern)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> write_lock(pattern_mutex);
|
||||
positive_pattern = new_pos_pattern;
|
||||
negative_pattern = new_neg_pattern;
|
||||
}
|
||||
}
|
||||
|
||||
std::pair<std::string, std::string> OwnFilteringChannel::safeGetPatterns()
|
||||
{
|
||||
std::shared_lock<std::shared_mutex> read_lock(pattern_mutex);
|
||||
return std::make_pair(positive_pattern, negative_pattern);
|
||||
}
|
||||
|
||||
void createOrUpdateFilterChannel(Poco::Logger & logger, const std::string & pos_pattern, const std::string & neg_pattern, Poco::AutoPtr<OwnPatternFormatter> pf, const std::string & name)
|
||||
{
|
||||
Poco::AutoPtr<Poco::Channel> src_channel(logger.getChannel(), true /*shared*/);
|
||||
Poco::AutoPtr<DB::OwnFilteringChannel> filter_channel(dynamic_cast<DB::OwnFilteringChannel*>(src_channel.get()), true);
|
||||
|
||||
// If this logger doesn't have it's own unique filter channel
|
||||
if (!filter_channel)
|
||||
{
|
||||
// Skip if regexp feature has never been used yet
|
||||
if (pos_pattern.empty() && neg_pattern.empty())
|
||||
return;
|
||||
|
||||
Poco::AutoPtr<DB::OwnFilteringChannel> new_filter_channel = new DB::OwnFilteringChannel(src_channel, pf, pos_pattern, neg_pattern, name);
|
||||
logger.setChannel(new_filter_channel);
|
||||
}
|
||||
// If logger has filter channel, but not it's own unique one (e.g copied from another by default), create copy
|
||||
else if (filter_channel->getAssignedLoggerName() != name)
|
||||
{
|
||||
Poco::AutoPtr<DB::OwnFilteringChannel> new_filter_channel = new DB::OwnFilteringChannel(filter_channel, pos_pattern, neg_pattern, name);
|
||||
logger.setChannel(new_filter_channel);
|
||||
}
|
||||
else
|
||||
{
|
||||
filter_channel->setRegexpPatterns(pos_pattern, neg_pattern);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
84
src/Loggers/OwnFilteringChannel.h
Normal file
84
src/Loggers/OwnFilteringChannel.h
Normal file
@ -0,0 +1,84 @@
|
||||
#pragma once
|
||||
#include <Poco/AutoPtr.h>
|
||||
#include <Poco/Channel.h>
|
||||
#include <Poco/Message.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Loggers/OwnPatternFormatter.h>
|
||||
#include <shared_mutex>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// Filters the logs based on regular expressions. Should be processed after formatting channel to read entire formatted text
|
||||
class OwnFilteringChannel : public Poco::Channel
|
||||
{
|
||||
public:
|
||||
explicit OwnFilteringChannel(Poco::AutoPtr<Poco::Channel> pChannel_, Poco::AutoPtr<OwnPatternFormatter> pf,
|
||||
const std::string & positive_pattern_, const std::string & negative_pattern_, const std::string & name_)
|
||||
: logger_name(name_), positive_pattern(positive_pattern_), negative_pattern(negative_pattern_), pChannel(pChannel_), pFormatter(pf)
|
||||
{
|
||||
}
|
||||
|
||||
explicit OwnFilteringChannel(Poco::AutoPtr<OwnFilteringChannel> other, const std::string & positive_pattern_, const std::string & negative_pattern_, const std::string & name_)
|
||||
: logger_name(name_), positive_pattern(positive_pattern_), negative_pattern(negative_pattern_), pChannel(other->pChannel), pFormatter(other->pFormatter)
|
||||
{
|
||||
}
|
||||
|
||||
// Only log if pass both positive and negative regexp checks.
|
||||
// Checks the regexps on the formatted text (without color), but then passes the raw text
|
||||
// to the split channel to handle formatting for individual channels (e.g apply color)
|
||||
void log(const Poco::Message & msg) override;
|
||||
|
||||
// Sets the regex patterns to use for filtering. Specifying an empty string pattern "" indicates no filtering
|
||||
void setRegexpPatterns(const std::string & new_pos_pattern, const std::string & new_neg_pattern);
|
||||
|
||||
std::string getAssignedLoggerName() const
|
||||
{
|
||||
return logger_name;
|
||||
}
|
||||
|
||||
void open() override
|
||||
{
|
||||
if (pChannel)
|
||||
pChannel->open();
|
||||
}
|
||||
|
||||
void close() override
|
||||
{
|
||||
if (pChannel)
|
||||
pChannel->close();
|
||||
}
|
||||
|
||||
void setProperty(const std::string & name, const std::string & value) override
|
||||
{
|
||||
if (pChannel)
|
||||
pChannel->setProperty(name, value);
|
||||
}
|
||||
|
||||
std::string getProperty(const std::string & name) const override
|
||||
{
|
||||
if (pChannel)
|
||||
return pChannel->getProperty(name);
|
||||
return "";
|
||||
}
|
||||
|
||||
private:
|
||||
bool regexpFilteredOut(const Poco::Message & msg);
|
||||
|
||||
// Create copy safely, so we don't have to worry about race conditions from reading and writing at the same time
|
||||
std::pair<std::string, std::string> safeGetPatterns();
|
||||
|
||||
const std::string logger_name;
|
||||
std::string positive_pattern;
|
||||
std::string negative_pattern;
|
||||
Poco::AutoPtr<Poco::Channel> pChannel;
|
||||
Poco::AutoPtr<OwnPatternFormatter> pFormatter;
|
||||
std::shared_mutex pattern_mutex;
|
||||
};
|
||||
|
||||
// Creates filter channel only if needed or updates if it already exists
|
||||
void createOrUpdateFilterChannel(Poco::Logger & logger, const std::string & pos_pattern, const std::string & neg_pattern, Poco::AutoPtr<OwnPatternFormatter> pf, const std::string & name = "");
|
||||
|
||||
}
|
0
tests/integration/test_regexp_logger/__init__.py
Normal file
0
tests/integration/test_regexp_logger/__init__.py
Normal file
6
tests/integration/test_regexp_logger/configs/log.xml
Normal file
6
tests/integration/test_regexp_logger/configs/log.xml
Normal file
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
</logger>
|
||||
</clickhouse>
|
74
tests/integration/test_regexp_logger/test.py
Normal file
74
tests/integration/test_regexp_logger/test.py
Normal file
@ -0,0 +1,74 @@
|
||||
import re
|
||||
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance(
|
||||
"node", with_zookeeper=False, main_configs=["configs/log.xml"]
|
||||
)
|
||||
|
||||
original_config = """
|
||||
<clickhouse>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
</logger>
|
||||
</clickhouse>
|
||||
"""
|
||||
|
||||
updated_config = """
|
||||
<clickhouse>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
<message_regexp_negative>.*Loaded config.*</message_regexp_negative>
|
||||
<message_regexps>
|
||||
<logger>
|
||||
<name>executeQuery</name>
|
||||
<message_regexp>.*Read.*</message_regexp>
|
||||
<message_regexp_negative>.*from.*</message_regexp_negative>
|
||||
</logger>
|
||||
</message_regexps>
|
||||
</logger>
|
||||
</clickhouse>
|
||||
"""
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def test_regexp_pattern_update(start_cluster):
|
||||
# Display config being used
|
||||
node.exec_in_container(["cat", "/etc/clickhouse-server/config.d/log.xml"])
|
||||
|
||||
# Make sure that there are enough log messages for the test
|
||||
for _ in range(5):
|
||||
node.query("SYSTEM RELOAD CONFIG")
|
||||
node.query("SELECT 1")
|
||||
|
||||
assert node.contains_in_log(r".*Loaded config.*")
|
||||
assert node.contains_in_log(r".*executeQuery.*Read.*")
|
||||
assert node.contains_in_log(r".*executeQuery.*from.*")
|
||||
|
||||
node.replace_config("/etc/clickhouse-server/config.d/log.xml", updated_config)
|
||||
node.query("SYSTEM RELOAD CONFIG;")
|
||||
node.rotate_logs()
|
||||
|
||||
for _ in range(5):
|
||||
node.query("SYSTEM RELOAD CONFIG")
|
||||
node.query("SELECT 1")
|
||||
|
||||
assert not node.contains_in_log(r".*Loaded config.*")
|
||||
assert node.contains_in_log(r".*executeQuery.*Read.*")
|
||||
assert not node.contains_in_log(r".*executeQuery.*from.*")
|
||||
|
||||
node.replace_config("/etc/clickhouse-server/config.d/log.xml", original_config)
|
Loading…
Reference in New Issue
Block a user