Add columns for values used in the formatting message into system.text_log

This commit is contained in:
Alexey Katsman 2024-02-05 23:43:39 +01:00 committed by Alex Katsman
parent 764c5436b5
commit ca4e2f581a
16 changed files with 167 additions and 34 deletions

View File

@ -66,9 +66,11 @@ public:
/// The thread and process ids are set.
Message(
const std::string & source, const std::string & text, Priority prio, const char * file, int line, std::string_view fmt_str = {});
const std::string & source, const std::string & text, Priority prio, const char * file, int line,
std::string_view fmt_str = {}, const std::vector<std::string> & fmt_str_args = {});
Message(
std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str);
std::string && source, std::string && text, Priority prio, const char * file, int line,
std::string_view fmt_str, std::vector<std::string> && fmt_str_args);
/// Creates a Message with the given source, text, priority,
/// source file path and line.
///
@ -161,6 +163,9 @@ public:
std::string_view getFormatString() const;
void setFormatString(std::string_view fmt_str);
const std::vector<std::string> & getFormatStringArgs() const;
void setFormatStringArgs(const std::vector<std::string> & fmt_str_args);
int getSourceLine() const;
/// Returns the source file line of the statement
/// generating the log message. May be 0
@ -210,6 +215,7 @@ private:
int _line;
StringMap * _pMap;
std::string_view _fmt_str;
std::vector<std::string> _fmt_str_args;
};

View File

@ -46,7 +46,9 @@ Message::Message(const std::string& source, const std::string& text, Priority pr
}
Message::Message(const std::string& source, const std::string& text, Priority prio, const char* file, int line, std::string_view fmt_str):
Message::Message(
const std::string& source, const std::string& text, Priority prio, const char* file, int line,
std::string_view fmt_str, const std::vector<std::string>& fmt_str_args):
_source(source),
_text(text),
_prio(prio),
@ -54,13 +56,16 @@ Message::Message(const std::string& source, const std::string& text, Priority pr
_file(file),
_line(line),
_pMap(0),
_fmt_str(fmt_str)
_fmt_str(fmt_str),
_fmt_str_args(fmt_str_args)
{
init();
}
Message::Message(std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str):
Message::Message(
std::string && source, std::string && text, Priority prio, const char * file, int line,
std::string_view fmt_str, std::vector<std::string> && fmt_str_args):
_source(std::move(source)),
_text(std::move(text)),
_prio(prio),
@ -68,7 +73,8 @@ Message::Message(std::string && source, std::string && text, Priority prio, cons
_file(file),
_line(line),
_pMap(0),
_fmt_str(fmt_str)
_fmt_str(fmt_str),
_fmt_str_args(std::move(fmt_str_args))
{
init();
}
@ -83,7 +89,8 @@ Message::Message(const Message& msg):
_pid(msg._pid),
_file(msg._file),
_line(msg._line),
_fmt_str(msg._fmt_str)
_fmt_str(msg._fmt_str),
_fmt_str_args(msg._fmt_str_args)
{
if (msg._pMap)
_pMap = new StringMap(*msg._pMap);
@ -102,7 +109,8 @@ Message::Message(const Message& msg, const std::string& text):
_pid(msg._pid),
_file(msg._file),
_line(msg._line),
_fmt_str(msg._fmt_str)
_fmt_str(msg._fmt_str),
_fmt_str_args(msg._fmt_str_args)
{
if (msg._pMap)
_pMap = new StringMap(*msg._pMap);
@ -154,6 +162,7 @@ void Message::swap(Message& msg)
swap(_line, msg._line);
swap(_pMap, msg._pMap);
swap(_fmt_str, msg._fmt_str);
swap(_fmt_str_args, msg._fmt_str_args);
}
@ -227,6 +236,17 @@ void Message::setFormatString(std::string_view fmt_str)
}
const std::vector<std::string>& Message::getFormatStringArgs() const
{
return _fmt_str_args;
}
void Message::setFormatStringArgs(const std::vector<std::string>& fmt_str_args)
{
_fmt_str_args = fmt_str_args;
}
bool Message::has(const std::string& param) const
{
return _pMap && (_pMap->find(param) != _pMap->end());

View File

@ -607,7 +607,8 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po
/// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons,
/// only the log will show the exact reason.
throw Exception(PreformattedMessage{message.str(),
"{}: Authentication failed: password is incorrect, or there is no user with such name.{}"},
"{}: Authentication failed: password is incorrect, or there is no user with such name.{}",
std::vector<std::string>{credentials.getUserName()}},
ErrorCodes::AUTHENTICATION_FAILED);
}
}

View File

@ -391,6 +391,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b
{
WriteBufferFromOwnString stream;
std::string_view message_format_string;
std::vector<std::string> message_format_string_args;
try
{
@ -402,6 +403,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
message_format_string = e.tryGetMessageFormatString();
message_format_string_args = e.getMessageFormatStringArgs();
}
catch (const Poco::Exception & e)
{
@ -462,7 +464,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b
catch (...) {} // NOLINT(bugprone-empty-catch)
}
return PreformattedMessage{stream.str(), message_format_string};
return PreformattedMessage{stream.str(), message_format_string, message_format_string_args};
}
@ -581,7 +583,7 @@ PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with
}
catch (...) {} // NOLINT(bugprone-empty-catch)
return PreformattedMessage{stream.str(), e.tryGetMessageFormatString()};
return PreformattedMessage{stream.str(), e.tryGetMessageFormatString(), e.getMessageFormatStringArgs()};
}
std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace)

View File

@ -13,6 +13,7 @@
#include <memory>
#include <vector>
#include <fmt/core.h>
#include <fmt/format.h>
#include <Poco/Exception.h>
@ -59,6 +60,7 @@ public:
std::terminate();
capture_thread_frame_pointers = thread_frame_pointers;
message_format_string = msg.format_string;
message_format_string_args = msg.format_string_args;
}
Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.text), code)
@ -67,6 +69,7 @@ public:
std::terminate();
capture_thread_frame_pointers = thread_frame_pointers;
message_format_string = msg.format_string;
message_format_string_args = msg.format_string_args;
}
/// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution
@ -107,12 +110,7 @@ public:
// Format message with fmt::format, like the logging functions.
template <typename... Args>
Exception(int code, FormatStringHelper<Args...> fmt, Args &&... args)
: Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code)
{
capture_thread_frame_pointers = thread_frame_pointers;
message_format_string = fmt.message_format_string;
}
Exception(int code, FormatStringHelper<Args...> fmt, Args &&... args) : Exception(fmt.format(std::forward<Args>(args)...), code) {}
struct CreateFromPocoTag {};
struct CreateFromSTDTag {};
@ -152,6 +150,8 @@ public:
std::string_view tryGetMessageFormatString() const { return message_format_string; }
std::vector<std::string> getMessageFormatStringArgs() const { return message_format_string_args; }
private:
#ifndef STD_EXCEPTION_HAS_STACK_TRACE
StackTrace trace;
@ -162,6 +162,7 @@ private:
protected:
std::string_view message_format_string;
std::vector<std::string> message_format_string_args;
/// Local copy of static per-thread thread_frame_pointers, should be mutable to be unpoisoned on printout
mutable std::vector<StackTrace::FramePointers> capture_thread_frame_pointers;
};
@ -193,26 +194,29 @@ public:
// Format message with fmt::format, like the logging functions.
template <typename... Args>
ErrnoException(int code, FormatStringHelper<Args...> fmt, Args &&... args)
: Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code), saved_errno(errno)
: Exception(fmt.format(std::forward<Args>(args)...), code), saved_errno(errno)
{
addMessage(", {}", errnoToString(saved_errno));
}
template <typename... Args>
ErrnoException(int code, int with_errno, FormatStringHelper<Args...> fmt, Args &&... args)
: Exception(fmt.format(std::forward<Args>(args)...), code), saved_errno(with_errno)
{
capture_thread_frame_pointers = thread_frame_pointers;
message_format_string = fmt.message_format_string;
addMessage(", {}", errnoToString(saved_errno));
}
template <typename... Args>
[[noreturn]] static void throwWithErrno(int code, int with_errno, FormatStringHelper<Args...> fmt, Args &&... args)
{
auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code, with_errno);
e.message_format_string = fmt.message_format_string;
auto e = ErrnoException(code, with_errno, std::move(fmt), std::forward<Args>(args)...);
throw e; /// NOLINT
}
template <typename... Args>
[[noreturn]] static void throwFromPath(int code, const std::string & path, FormatStringHelper<Args...> fmt, Args &&... args)
{
auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code, errno);
e.message_format_string = fmt.message_format_string;
auto e = ErrnoException(code, errno, std::move(fmt), std::forward<Args>(args)...);
e.path = path;
throw e; /// NOLINT
}
@ -221,8 +225,7 @@ public:
[[noreturn]] static void
throwFromPathWithErrno(int code, const std::string & path, int with_errno, FormatStringHelper<Args...> fmt, Args &&... args)
{
auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code, with_errno);
e.message_format_string = fmt.message_format_string;
auto e = ErrnoException(code, with_errno, std::move(fmt), std::forward<Args>(args)...);
e.path = path;
throw e; /// NOLINT
}

View File

@ -2,8 +2,11 @@
#include <base/defines.h>
#include <base/types.h>
#include <fmt/args.h>
#include <fmt/core.h>
#include <fmt/format.h>
#include <mutex>
#include <type_traits>
#include <unordered_map>
#include <Poco/Logger.h>
#include <Poco/Message.h>
@ -14,6 +17,10 @@ struct PreformattedMessage;
consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs);
template <typename T> constexpr std::string_view tryGetStaticFormatString(T && x);
[[maybe_unused]] inline void tryGetFormattedArgs(std::vector<std::string>&) {};
template <typename T, typename... Ts> [[maybe_unused]] inline void tryGetFormattedArgs(std::vector<std::string>&, T &&, Ts && ...);
template <typename... Args> inline std::string tryGetArgsAndFormat(std::vector<std::string>&, fmt::format_string<Args...>, Args && ...);
/// Extract format string from a string literal and constructs consteval fmt::format_string
template <typename... Args>
struct FormatStringHelperImpl
@ -39,6 +46,7 @@ struct PreformattedMessage
{
std::string text;
std::string_view format_string;
std::vector<std::string> format_string_args;
template <typename... Args>
static PreformattedMessage create(FormatStringHelper<Args...> fmt, Args &&... args);
@ -47,22 +55,26 @@ struct PreformattedMessage
operator std::string () && { return std::move(text); } /// NOLINT
operator fmt::format_string<> () const { UNREACHABLE(); } /// NOLINT
void apply(std::string & out_text, std::string_view & out_format_string) const &
void apply(std::string & out_text, std::string_view & out_format_string, std::vector<std::string> & out_format_string_args) const &
{
out_text = text;
out_format_string = format_string;
out_format_string_args = format_string_args;
}
void apply(std::string & out_text, std::string_view & out_format_string) &&
void apply(std::string & out_text, std::string_view & out_format_string, std::vector<std::string> & out_format_string_args) &&
{
out_text = std::move(text);
out_format_string = format_string;
out_format_string_args = std::move(format_string_args);
}
};
template <typename... Args>
PreformattedMessage FormatStringHelperImpl<Args...>::format(Args && ...args) const
{
return PreformattedMessage{fmt::format(fmt_str, std::forward<Args>(args)...), message_format_string};
std::vector<std::string> out_format_string_args;
std::string msg_text = tryGetArgsAndFormat(out_format_string_args, fmt_str, std::forward<Args>(args)...);
return PreformattedMessage{msg_text, message_format_string, out_format_string_args};
}
template <typename... Args>
@ -113,12 +125,23 @@ template <typename T> constexpr std::string_view tryGetStaticFormatString(T && x
}
}
template <typename T, typename... Ts> void tryGetFormattedArgs(std::vector<std::string>& out, T && x, Ts && ...rest)
{
if constexpr (std::is_base_of_v<fmt::detail::view, std::decay_t<T>>)
out.push_back(fmt::format("{}", std::remove_reference_t<T>(x)));
else
out.push_back(fmt::format("{}", std::forward<T>(x)));
tryGetFormattedArgs(out, std::forward<Ts>(rest)...);
}
/// Constexpr ifs are not like ifdefs, and compiler still checks that unneeded code can be compiled
/// This template is useful to avoid compilation failures when condition of some "constexpr if" is false
template<bool enable> struct ConstexprIfsAreNotIfdefs
{
template <typename T> constexpr static std::string_view getStaticFormatString(T &&) { return {}; }
template <typename T> static PreformattedMessage getPreformatted(T &&) { return {}; }
template <typename... Args> static std::string getArgsAndFormat(std::vector<std::string>&, fmt::format_string<Args...>, Args &&...) { return {}; }
};
template<> struct ConstexprIfsAreNotIfdefs<true>
@ -133,8 +156,19 @@ template<> struct ConstexprIfsAreNotIfdefs<true>
}
template <typename T> static T && getPreformatted(T && x) { return std::forward<T>(x); }
template <typename... Args> static std::string getArgsAndFormat(std::vector<std::string>& out, fmt::format_string<Args...> fmt_str, Args && ...args)
{
return tryGetArgsAndFormat(out, std::move(fmt_str), std::forward<Args>(args)...);
}
};
template <typename... Args> inline std::string tryGetArgsAndFormat(std::vector<std::string>& out, fmt::format_string<Args...> fmt_str, Args && ...args)
{
tryGetFormattedArgs(out, args...);
return fmt::format(fmt_str, std::forward<Args>(args)...);
}
template <typename... Ts> constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); }
template <typename T, typename... Ts> constexpr auto firstArg(T && x, Ts &&...) { return std::forward<T>(x); }
/// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor

View File

@ -2,6 +2,7 @@
/// Macros for convenient usage of Poco logger.
#include <unistd.h>
#include <fmt/args.h>
#include <fmt/format.h>
#include <Poco/Logger.h>
#include <Poco/Message.h>
@ -80,6 +81,7 @@ namespace impl
\
std::string_view _format_string; \
std::string _formatted_message; \
std::vector<std::string> _format_string_args; \
\
if constexpr (LogTypeInfo::is_static) \
{ \
@ -91,17 +93,17 @@ namespace impl
if constexpr (is_preformatted_message) \
{ \
static_assert(_nargs == 1 || !is_preformatted_message); \
ConstexprIfsAreNotIfdefs<is_preformatted_message>::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string); \
ConstexprIfsAreNotIfdefs<is_preformatted_message>::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string, _format_string_args); \
} \
else \
{ \
_formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : fmt::format(__VA_ARGS__); \
_formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : ConstexprIfsAreNotIfdefs<!is_preformatted_message>::getArgsAndFormat(_format_string_args, __VA_ARGS__); \
} \
\
std::string _file_function = __FILE__ "; "; \
_file_function += __PRETTY_FUNCTION__; \
Poco::Message _poco_message(_logger->name(), std::move(_formatted_message), \
(PRIORITY), _file_function.c_str(), __LINE__, _format_string); \
(PRIORITY), _file_function.c_str(), __LINE__, _format_string, _format_string_args); \
_channel->log(_poco_message); \
} \
catch (const Poco::Exception & logger_exception) \

View File

@ -794,7 +794,7 @@ inline bool tryParseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer &
if (isNativeNumber(result_type) && !(result_type.getName() == "IPv4" || result_type.getName() == "IPv6"))
message_buf << ". Note: there are to" << result_type.getName() << "OrZero and to" << result_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception.";
throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}"}, ErrorCodes::CANNOT_PARSE_TEXT);
throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}", {String(read_buffer.buffer().begin(), read_buffer.buffer().size()), result_type.getName()}}, ErrorCodes::CANNOT_PARSE_TEXT);
}

View File

@ -86,6 +86,7 @@ struct QueryLogElement
String exception;
String stack_trace;
std::string_view exception_format_string{};
std::vector<std::string> exception_format_string_args{};
ClientInfo client_info;

View File

@ -53,6 +53,16 @@ ColumnsDescription TextLogElement::getColumnsDescription()
{"source_line", std::make_shared<DataTypeUInt64>(), "Source line from which the logging was done."},
{"message_format_string", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "A format string that was used to format the message."},
{"value1", std::make_shared<DataTypeString>(), "Argument 1 that was used to format the message."},
{"value2", std::make_shared<DataTypeString>(), "Argument 2 that was used to format the message."},
{"value3", std::make_shared<DataTypeString>(), "Argument 3 that was used to format the message."},
{"value4", std::make_shared<DataTypeString>(), "Argument 4 that was used to format the message."},
{"value5", std::make_shared<DataTypeString>(), "Argument 5 that was used to format the message."},
{"value6", std::make_shared<DataTypeString>(), "Argument 6 that was used to format the message."},
{"value7", std::make_shared<DataTypeString>(), "Argument 7 that was used to format the message."},
{"value8", std::make_shared<DataTypeString>(), "Argument 8 that was used to format the message."},
{"value9", std::make_shared<DataTypeString>(), "Argument 9 that was used to format the message."},
{"value10", std::make_shared<DataTypeString>(), "Argument 10 that was used to format the message."},
};
}
@ -79,6 +89,16 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(source_line);
columns[i++]->insert(message_format_string);
columns[i++]->insert(value1);
columns[i++]->insert(value2);
columns[i++]->insert(value3);
columns[i++]->insert(value4);
columns[i++]->insert(value5);
columns[i++]->insert(value6);
columns[i++]->insert(value7);
columns[i++]->insert(value8);
columns[i++]->insert(value9);
columns[i++]->insert(value10);
}
TextLog::TextLog(ContextPtr context_,

View File

@ -29,6 +29,16 @@ struct TextLogElement
UInt64 source_line{};
std::string_view message_format_string;
String value1;
String value2;
String value3;
String value4;
String value5;
String value6;
String value7;
String value8;
String value9;
String value10;
static std::string name() { return "TextLog"; }
static ColumnsDescription getColumnsDescription();

View File

@ -1249,7 +1249,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
if (no_throw)
return false;
throw Exception(PreformattedMessage{ss.str(), format_string}, ErrorCodes::UNKNOWN_IDENTIFIER);
throw Exception(PreformattedMessage{ss.str(), format_string, std::vector<std::string>{}}, ErrorCodes::UNKNOWN_IDENTIFIER);
}
required_source_columns.swap(source_columns);

View File

@ -200,6 +200,7 @@ static void logException(ContextPtr context, QueryLogElement & elem, bool log_er
/// so we pass elem.exception_format_string as format string instead.
PreformattedMessage message;
message.format_string = elem.exception_format_string;
message.format_string_args = elem.exception_format_string_args;
if (elem.stack_trace.empty() || !log_error)
message.text = fmt::format("{} (from {}){} (in query: {})", elem.exception,
@ -504,6 +505,7 @@ void logQueryException(
auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false);
elem.exception = std::move(exception_message.text);
elem.exception_format_string = exception_message.format_string;
elem.exception_format_string_args = exception_message.format_string_args;
QueryStatusPtr process_list_elem = context->getProcessListElement();
@ -597,6 +599,7 @@ void logExceptionBeforeStart(
auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false);
elem.exception = std::move(exception_message.text);
elem.exception_format_string = exception_message.format_string;
elem.exception_format_string_args = exception_message.format_string_args;
elem.client_info = context->getClientInfo();

View File

@ -131,6 +131,21 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
elem.source_line = msg.getSourceLine();
elem.message_format_string = msg.getFormatString();
#define SET_VALUE_IF_EXISTS(INDEX) if ((INDEX) <= msg.getFormatStringArgs().size()) (elem.value##INDEX) = msg.getFormatStringArgs()[(INDEX) - 1]
SET_VALUE_IF_EXISTS(1);
SET_VALUE_IF_EXISTS(2);
SET_VALUE_IF_EXISTS(3);
SET_VALUE_IF_EXISTS(4);
SET_VALUE_IF_EXISTS(5);
SET_VALUE_IF_EXISTS(6);
SET_VALUE_IF_EXISTS(7);
SET_VALUE_IF_EXISTS(8);
SET_VALUE_IF_EXISTS(9);
SET_VALUE_IF_EXISTS(10);
#undef SET_VALUE_IF_EXISTS
std::shared_ptr<SystemLogQueue<TextLogElement>> text_log_locked{};
text_log_locked = text_log.lock();
if (text_log_locked)

View File

@ -0,0 +1,13 @@
set allow_experimental_analyzer = true;
select count; -- { serverError 47 }
select conut(); -- { serverError 46 }
system flush logs;
select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB';
select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier \'{}\' in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count';
select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name \'{}\' does not exists. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%';