Merge branch 'master' into trivial-merge-selector

This commit is contained in:
Alexey Milovidov 2024-10-23 22:25:06 +02:00
commit b764db6177
14 changed files with 399 additions and 23 deletions

View File

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

View File

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

View File

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

View File

@ -1079,6 +1079,8 @@ Possible values:
Default value: 0 bytes.
Note that if both `min_free_disk_bytes_to_perform_insert` and `min_free_disk_ratio_to_perform_insert` are specified, ClickHouse will count on the value that will allow to perform inserts on a bigger amount of free memory.
## min_free_disk_ratio_to_perform_insert
The minimum free to total disk space ratio to perform an `INSERT`. Must be a floating point value between 0 and 1. Note that this setting:

View File

@ -1124,15 +1124,15 @@ Strings Context::getWarnings() const
SharedLockGuard lock(shared->mutex);
common_warnings = shared->warnings;
if (CurrentMetrics::get(CurrentMetrics::AttachedTable) > static_cast<Int64>(shared->max_table_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of attached tables is more than {}", shared->max_table_num_to_warn));
common_warnings.emplace_back(fmt::format("The number of attached tables is more than {}.", shared->max_table_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::AttachedView) > static_cast<Int64>(shared->max_view_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of attached views is more than {}", shared->max_view_num_to_warn));
common_warnings.emplace_back(fmt::format("The number of attached views is more than {}.", shared->max_view_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::AttachedDictionary) > static_cast<Int64>(shared->max_dictionary_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of attached dictionaries is more than {}", shared->max_dictionary_num_to_warn));
common_warnings.emplace_back(fmt::format("The number of attached dictionaries is more than {}.", shared->max_dictionary_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::AttachedDatabase) > static_cast<Int64>(shared->max_database_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of attached databases is more than {}", shared->max_database_num_to_warn));
common_warnings.emplace_back(fmt::format("The number of attached databases is more than {}.", shared->max_database_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::PartsActive) > static_cast<Int64>(shared->max_part_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of active parts is more than {}", shared->max_part_num_to_warn));
common_warnings.emplace_back(fmt::format("The number of active parts is more than {}.", shared->max_part_num_to_warn));
}
/// Make setting's name ordered
auto obsolete_settings = settings->getChangedAndObsoleteNames();

View File

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

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

View 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 = "");
}

View File

@ -0,0 +1,6 @@
<clickhouse>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
</logger>
</clickhouse>

View 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)

View File

@ -1,5 +1,5 @@
The number of attached tables is more than 5
The number of attached views is more than 5
The number of attached dictionaries is more than 5
The number of attached databases is more than 2
The number of active parts is more than 10
The number of attached tables is more than 5.
The number of attached views is more than 5.
The number of attached dictionaries is more than 5.
The number of attached databases is more than 2.
The number of active parts is more than 10.

View File

@ -71,11 +71,11 @@ INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_10 VALUES (1, 'Hello
INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_11 VALUES (1, 'Hello');
SELECT * FROM system.warnings where message in (
'The number of attached tables is more than 5',
'The number of attached views is more than 5',
'The number of attached dictionaries is more than 5',
'The number of attached databases is more than 2',
'The number of active parts is more than 10'
'The number of attached tables is more than 5.',
'The number of attached views is more than 5.',
'The number of attached dictionaries is more than 5.',
'The number of attached databases is more than 2.',
'The number of active parts is more than 10.'
);
DROP DATABASE IF EXISTS test_max_num_to_warn_02931;

View File

@ -55,7 +55,7 @@ KEEP_WORKTREE=0
#
if [ "$1" == "--help" ] || [ -z "$1" ]; then usage; fi
while getopts "vc:CIi:l:bkB:t:o:O:g:" OPT; do
while getopts "vc:CIi:l:b:kB:t:o:O:g:" OPT; do
case "$OPT" in
v) set -x; ;;
c) CXX="$OPTARG"; ;;