mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #39161 from amosbird/text-log-source-regexp
Control send logs behavior with source name regexp
This commit is contained in:
commit
5cb3b25754
@ -391,6 +391,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(Bool, log_query_views, true, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \
|
||||
M(String, log_comment, "", "Log comment into system.query_log table and server log. It can be set to arbitrary string no longer than max_query_size.", 0) \
|
||||
M(LogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
|
||||
M(String, send_logs_source_regexp, "", "Send server text logs with specified regexp to match log source name. Empty means all sources.", 0) \
|
||||
M(Bool, enable_optimize_predicate_expression, true, "If it is set to true, optimize predicates to subqueries.", 0) \
|
||||
M(Bool, enable_optimize_predicate_expression_to_final_subquery, true, "Allow push predicate to final subquery.", 0) \
|
||||
M(Bool, allow_push_predicate_when_subquery_contains_with, true, "Allows push predicate when subquery contains WITH clause", 0) \
|
||||
|
@ -66,4 +66,19 @@ const char * InternalTextLogsQueue::getPriorityName(int priority)
|
||||
return (priority >= 1 && priority <= 8) ? PRIORITIES[priority] : PRIORITIES[0];
|
||||
}
|
||||
|
||||
bool InternalTextLogsQueue::isNeeded(int priority, const String & source) const
|
||||
{
|
||||
bool is_needed = priority <= max_priority;
|
||||
|
||||
if (is_needed && source_regexp)
|
||||
is_needed = re2::RE2::PartialMatch(source, *source_regexp);
|
||||
|
||||
return is_needed;
|
||||
}
|
||||
|
||||
void InternalTextLogsQueue::setSourceRegexp(const String & regexp)
|
||||
{
|
||||
source_regexp = std::make_unique<re2::RE2>(regexp);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/OvercommitTracker.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
#include <re2/re2.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -15,6 +15,8 @@ public:
|
||||
|
||||
InternalTextLogsQueue();
|
||||
|
||||
bool isNeeded(int priority, const String & source) const;
|
||||
|
||||
static Block getSampleBlock();
|
||||
static MutableColumns getSampleColumns();
|
||||
|
||||
@ -23,6 +25,11 @@ public:
|
||||
|
||||
/// Converts priority from Poco::Message::Priority to a string
|
||||
static const char * getPriorityName(int priority);
|
||||
|
||||
void setSourceRegexp(const String & regexp);
|
||||
private:
|
||||
/// If not null, you should only push logs which are matched with this regexp
|
||||
std::unique_ptr<re2::RE2> source_regexp;
|
||||
};
|
||||
|
||||
using InternalTextLogsQueuePtr = std::shared_ptr<InternalTextLogsQueue>;
|
||||
|
@ -24,7 +24,7 @@ void OwnSplitChannel::log(const Poco::Message & msg)
|
||||
#ifdef WITH_TEXT_LOG
|
||||
auto logs_queue = CurrentThread::getInternalTextLogsQueue();
|
||||
|
||||
if (channels.empty() && (logs_queue == nullptr || msg.getPriority() > logs_queue->max_priority))
|
||||
if (channels.empty() && (logs_queue == nullptr || !logs_queue->isNeeded(msg.getPriority(), msg.getSource())))
|
||||
return;
|
||||
#endif
|
||||
|
||||
@ -93,7 +93,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
|
||||
auto logs_queue = CurrentThread::getInternalTextLogsQueue();
|
||||
|
||||
/// Log to "TCP queue" if message is not too noisy
|
||||
if (logs_queue && msg.getPriority() <= logs_queue->max_priority)
|
||||
if (logs_queue && logs_queue->isNeeded(msg.getPriority(), msg.getSource()))
|
||||
{
|
||||
MutableColumns columns = InternalTextLogsQueue::getSampleColumns();
|
||||
|
||||
|
@ -848,6 +848,7 @@ namespace
|
||||
{
|
||||
logs_queue = std::make_shared<InternalTextLogsQueue>();
|
||||
logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString());
|
||||
logs_queue->setSourceRegexp(settings.send_logs_source_regexp);
|
||||
CurrentThread::attachInternalTextLogsQueue(logs_queue, client_logs_level);
|
||||
CurrentThread::setFatalErrorCallback([this]{ onFatalError(); });
|
||||
}
|
||||
|
@ -241,6 +241,7 @@ void TCPHandler::runImpl()
|
||||
{
|
||||
state.logs_queue = std::make_shared<InternalTextLogsQueue>();
|
||||
state.logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString());
|
||||
state.logs_queue->setSourceRegexp(query_context->getSettingsRef().send_logs_source_regexp);
|
||||
CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level);
|
||||
CurrentThread::setFatalErrorCallback([this]
|
||||
{
|
||||
|
@ -0,0 +1 @@
|
||||
1
|
11
tests/queries/0_stateless/02359_send_logs_source_regexp.sh
Executable file
11
tests/queries/0_stateless/02359_send_logs_source_regexp.sh
Executable file
@ -0,0 +1,11 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
[ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED
|
||||
|
||||
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g')
|
||||
regexp="executeQuery|InterpreterSelectQuery"
|
||||
$CLICKHOUSE_CLIENT --send_logs_source_regexp "$regexp" -q "SELECT 1;" 2> >(grep -v -E "$regexp" 1>&2)
|
Loading…
Reference in New Issue
Block a user