mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #29586 from evillique/log_levels_update
Add log levels updates
This commit is contained in:
commit
f4bfed9d3a
@ -84,7 +84,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
|
||||
Poco::AutoPtr<DB::OwnFormattingChannel> log = new DB::OwnFormattingChannel(pf, log_file);
|
||||
log->setLevel(log_level);
|
||||
split->addChannel(log);
|
||||
split->addChannel(log, "log");
|
||||
}
|
||||
|
||||
const auto errorlog_path = config.getString("logger.errorlog", "");
|
||||
@ -116,7 +116,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
Poco::AutoPtr<DB::OwnFormattingChannel> errorlog = new DB::OwnFormattingChannel(pf, error_log_file);
|
||||
errorlog->setLevel(errorlog_level);
|
||||
errorlog->open();
|
||||
split->addChannel(errorlog);
|
||||
split->addChannel(errorlog, "errorlog");
|
||||
}
|
||||
|
||||
if (config.getBool("logger.use_syslog", false))
|
||||
@ -155,7 +155,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
Poco::AutoPtr<DB::OwnFormattingChannel> log = new DB::OwnFormattingChannel(pf, syslog_channel);
|
||||
log->setLevel(syslog_level);
|
||||
|
||||
split->addChannel(log);
|
||||
split->addChannel(log, "syslog");
|
||||
}
|
||||
|
||||
bool should_log_to_console = isatty(STDIN_FILENO) || isatty(STDERR_FILENO);
|
||||
@ -177,7 +177,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
Poco::AutoPtr<DB::OwnFormattingChannel> log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel);
|
||||
logger.warning("Logging " + console_log_level_string + " to console");
|
||||
log->setLevel(console_log_level);
|
||||
split->addChannel(log);
|
||||
split->addChannel(log, "console");
|
||||
}
|
||||
|
||||
split->open();
|
||||
@ -224,6 +224,89 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
}
|
||||
}
|
||||
|
||||
void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger)
|
||||
{
|
||||
int max_log_level = 0;
|
||||
|
||||
const auto log_level_string = config.getString("logger.level", "trace");
|
||||
int log_level = Poco::Logger::parseLevel(log_level_string);
|
||||
if (log_level > max_log_level)
|
||||
max_log_level = log_level;
|
||||
|
||||
const auto log_path = config.getString("logger.log", "");
|
||||
if (!log_path.empty())
|
||||
split->setLevel("log", log_level);
|
||||
else
|
||||
split->setLevel("log", 0);
|
||||
|
||||
// Set level to console
|
||||
bool is_daemon = config.getBool("application.runAsDaemon", false);
|
||||
bool should_log_to_console = isatty(STDIN_FILENO) || isatty(STDERR_FILENO);
|
||||
if (config.getBool("logger.console", false)
|
||||
|| (!config.hasProperty("logger.console") && !is_daemon && should_log_to_console))
|
||||
split->setLevel("console", log_level);
|
||||
else
|
||||
split->setLevel("console", 0);
|
||||
|
||||
// Set level to errorlog
|
||||
int errorlog_level = 0;
|
||||
const auto errorlog_path = config.getString("logger.errorlog", "");
|
||||
if (!errorlog_path.empty())
|
||||
{
|
||||
errorlog_level = Poco::Logger::parseLevel(config.getString("logger.errorlog_level", "notice"));
|
||||
if (errorlog_level > max_log_level)
|
||||
max_log_level = errorlog_level;
|
||||
}
|
||||
split->setLevel("errorlog", errorlog_level);
|
||||
|
||||
// Set level to syslog
|
||||
int syslog_level = 0;
|
||||
if (config.getBool("logger.use_syslog", false))
|
||||
{
|
||||
syslog_level = Poco::Logger::parseLevel(config.getString("logger.syslog_level", log_level_string));
|
||||
if (syslog_level > max_log_level)
|
||||
max_log_level = syslog_level;
|
||||
}
|
||||
split->setLevel("syslog", syslog_level);
|
||||
|
||||
// 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);
|
||||
for (const auto & name : names)
|
||||
logger.root().get(name).setLevel(max_log_level);
|
||||
|
||||
logger.root().setLevel(max_log_level);
|
||||
|
||||
// Explicitly specified log levels for specific loggers.
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys loggers_level;
|
||||
config.keys("logger.levels", loggers_level);
|
||||
|
||||
if (!loggers_level.empty())
|
||||
{
|
||||
for (const auto & key : loggers_level)
|
||||
{
|
||||
if (key == "logger" || key.starts_with("logger["))
|
||||
{
|
||||
const std::string name(config.getString("logger.levels." + key + ".name"));
|
||||
const std::string level(config.getString("logger.levels." + key + ".level"));
|
||||
logger.root().get(name).setLevel(level);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Legacy syntax
|
||||
const std::string level(config.getString("logger.levels." + key, "trace"));
|
||||
logger.root().get(key).setLevel(level);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void Loggers::closeLogs(Poco::Logger & logger)
|
||||
{
|
||||
if (log_file)
|
||||
|
@ -19,6 +19,8 @@ class Loggers
|
||||
public:
|
||||
void buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger, const std::string & cmd_name = "");
|
||||
|
||||
void updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger);
|
||||
|
||||
/// Close log files. On next log write files will be reopened.
|
||||
void closeLogs(Poco::Logger & logger);
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#pragma once
|
||||
#include <atomic>
|
||||
#include <Poco/AutoPtr.h>
|
||||
#include <Poco/Channel.h>
|
||||
#include <Poco/FormattingChannel.h>
|
||||
@ -14,7 +15,7 @@ class OwnFormattingChannel : public Poco::Channel, public ExtendedLogChannel
|
||||
public:
|
||||
explicit OwnFormattingChannel(
|
||||
Poco::AutoPtr<OwnPatternFormatter> pFormatter_ = nullptr, Poco::AutoPtr<Poco::Channel> pChannel_ = nullptr)
|
||||
: pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_))
|
||||
: pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)), priority(Poco::Message::PRIO_TRACE)
|
||||
{
|
||||
}
|
||||
|
||||
@ -45,7 +46,7 @@ public:
|
||||
private:
|
||||
Poco::AutoPtr<OwnPatternFormatter> pFormatter;
|
||||
Poco::AutoPtr<Poco::Channel> pChannel;
|
||||
Poco::Message::Priority priority = Poco::Message::PRIO_TRACE;
|
||||
std::atomic<Poco::Message::Priority> priority;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "OwnSplitChannel.h"
|
||||
#include "OwnFormattingChannel.h"
|
||||
|
||||
#include <iostream>
|
||||
#include <Core/Block.h>
|
||||
@ -75,7 +76,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
|
||||
ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg);
|
||||
|
||||
/// Log data to child channels
|
||||
for (auto & channel : channels)
|
||||
for (auto & [name, channel] : channels)
|
||||
{
|
||||
if (channel.second)
|
||||
channel.second->logExtended(msg_ext); // extended child
|
||||
@ -137,9 +138,9 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
|
||||
}
|
||||
|
||||
|
||||
void OwnSplitChannel::addChannel(Poco::AutoPtr<Poco::Channel> channel)
|
||||
void OwnSplitChannel::addChannel(Poco::AutoPtr<Poco::Channel> channel, const std::string & name)
|
||||
{
|
||||
channels.emplace_back(std::move(channel), dynamic_cast<ExtendedLogChannel *>(channel.get()));
|
||||
channels.emplace(name, ExtendedChannelPtrPair(std::move(channel), dynamic_cast<ExtendedLogChannel *>(channel.get())));
|
||||
}
|
||||
|
||||
void OwnSplitChannel::addTextLog(std::shared_ptr<DB::TextLog> log, int max_priority)
|
||||
@ -149,4 +150,14 @@ void OwnSplitChannel::addTextLog(std::shared_ptr<DB::TextLog> log, int max_prior
|
||||
text_log_max_priority.store(max_priority, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
void OwnSplitChannel::setLevel(const std::string & name, int level)
|
||||
{
|
||||
auto it = channels.find(name);
|
||||
if (it != channels.end())
|
||||
{
|
||||
if (auto * channel = dynamic_cast<DB::OwnFormattingChannel *>(it->second.first.get()))
|
||||
channel->setLevel(level);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -18,10 +18,12 @@ public:
|
||||
/// Makes an extended message from msg and passes it to the client logs queue and child (if possible)
|
||||
void log(const Poco::Message & msg) override;
|
||||
/// Adds a child channel
|
||||
void addChannel(Poco::AutoPtr<Poco::Channel> channel);
|
||||
void addChannel(Poco::AutoPtr<Poco::Channel> channel, const std::string & name);
|
||||
|
||||
void addTextLog(std::shared_ptr<DB::TextLog> log, int max_priority);
|
||||
|
||||
void setLevel(const std::string & name, int level);
|
||||
|
||||
private:
|
||||
void logSplit(const Poco::Message & msg);
|
||||
void tryLogSplit(const Poco::Message & msg);
|
||||
@ -29,7 +31,7 @@ private:
|
||||
using ChannelPtr = Poco::AutoPtr<Poco::Channel>;
|
||||
/// Handler and its pointer casted to extended interface
|
||||
using ExtendedChannelPtrPair = std::pair<ChannelPtr, ExtendedLogChannel *>;
|
||||
std::vector<ExtendedChannelPtrPair> channels;
|
||||
std::map<std::string, ExtendedChannelPtrPair> channels;
|
||||
|
||||
std::mutex text_log_mutex;
|
||||
|
||||
|
@ -844,7 +844,7 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
// FIXME logging-related things need synchronization -- see the 'Logger * log' saved
|
||||
// in a lot of places. For now, disable updating log configuration without server restart.
|
||||
//setTextLog(global_context->getTextLog());
|
||||
//buildLoggers(*config, logger());
|
||||
updateLevels(*config, logger());
|
||||
global_context->setClustersConfig(config);
|
||||
global_context->setMacros(std::make_unique<Macros>(*config, "macros", log));
|
||||
global_context->setExternalAuthenticatorsConfig(*config);
|
||||
|
6
tests/integration/test_log_levels_update/configs/log.xml
Normal file
6
tests/integration/test_log_levels_update/configs/log.xml
Normal file
@ -0,0 +1,6 @@
|
||||
<yandex>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
</logger>
|
||||
</yandex>
|
50
tests/integration/test_log_levels_update/test.py
Normal file
50
tests/integration/test_log_levels_update/test.py
Normal file
@ -0,0 +1,50 @@
|
||||
import pytest
|
||||
import re
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__, name="log_quries_probability")
|
||||
node = cluster.add_instance('node', with_zookeeper=False)
|
||||
|
||||
config = '''<yandex>
|
||||
<logger>
|
||||
<level>information</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
</logger>
|
||||
</yandex>'''
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def get_log(node):
|
||||
return node.exec_in_container(["bash", "-c", "cat /var/log/clickhouse-server/clickhouse-server.log"])
|
||||
|
||||
def test_log_levels_update(start_cluster):
|
||||
# Make sure that there are enough log messages for the test
|
||||
for i in range(5):
|
||||
node.query("SELECT 1")
|
||||
|
||||
log = get_log(node)
|
||||
assert re.search("(<Trace>|<Debug>)", log)
|
||||
|
||||
node.replace_config("/etc/clickhouse-server/config.d/log.xml", config)
|
||||
node.query("SYSTEM RELOAD CONFIG;")
|
||||
node.exec_in_container(["bash", "-c", "> /var/log/clickhouse-server/clickhouse-server.log"])
|
||||
|
||||
for i in range(5):
|
||||
node.query("SELECT 1")
|
||||
|
||||
log = get_log(node)
|
||||
assert len(log) > 0
|
||||
assert not re.search("(<Trace>|<Debug>)", log)
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user