Skip to content

Commit

Permalink
Merge pull request ClickHouse#45342 from ClickHouse/exception_message…
Browse files Browse the repository at this point in the history
…_patterns

Save message format strings for DB::Exception
  • Loading branch information
tavplubix authored Jan 20, 2023
2 parents 136e4ec + ec5d7d0 commit 910d6dc
Show file tree
Hide file tree
Showing 67 changed files with 380 additions and 304 deletions.
2 changes: 1 addition & 1 deletion contrib/poco
3 changes: 2 additions & 1 deletion docker/test/stateless/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -128,9 +128,10 @@ function run_tests()

if [[ "${HIGH_LEVEL_COVERAGE}" = "YES" ]]; then
ADDITIONAL_OPTIONS+=('--report-coverage')
ADDITIONAL_OPTIONS+=('--report-logs-stats')
fi

ADDITIONAL_OPTIONS+=('--report-logs-stats')

set +e
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
Expand Down
4 changes: 3 additions & 1 deletion src/Access/AccessControl.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -575,7 +575,9 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne

/// 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(message.str(), ErrorCodes::AUTHENTICATION_FAILED);
throw Exception(PreformattedMessage{message.str(),
"{}: Authentication failed: password is incorrect, or there is no user with such name.{}"},
ErrorCodes::AUTHENTICATION_FAILED);
}
}

Expand Down
3 changes: 1 addition & 2 deletions src/AggregateFunctions/AggregateFunctionBoundingRatio.h
Original file line number Diff line number Diff line change
Expand Up @@ -118,8 +118,7 @@ class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper
const auto * y_arg = arguments.at(1).get();

if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.",
ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal types of arguments of aggregate function {}, must have number representation.", getName());
}

bool allocatesMemoryInArena() const override { return false; }
Expand Down
2 changes: 1 addition & 1 deletion src/AggregateFunctions/AggregateFunctionMannWhitney.h
Original file line number Diff line number Diff line change
Expand Up @@ -226,7 +226,7 @@ class AggregateFunctionMannWhitney final:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if (!this->data(place).size_x || !this->data(place).size_y)
throw Exception("Aggregate function " + getName() + " require both samples to be non empty", ErrorCodes::BAD_ARGUMENTS);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());

auto [u_statistic, p_value] = this->data(place).getResult(alternative, continuity_correction);

Expand Down
4 changes: 2 additions & 2 deletions src/Backups/BackupEntriesCollector.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ BackupEntries BackupEntriesCollector::run()

Strings BackupEntriesCollector::setStage(const String & new_stage, const String & message)
{
LOG_TRACE(log, "{}", toUpperFirst(new_stage));
LOG_TRACE(log, fmt::runtime(toUpperFirst(new_stage)));
current_stage = new_stage;

backup_coordination->setStage(backup_settings.host_id, new_stage, message);
Expand Down Expand Up @@ -215,7 +215,7 @@ void BackupEntriesCollector::gatherMetadataAndCheckConsistency()
if (std::chrono::steady_clock::now() > consistent_metadata_snapshot_end_time)
inconsistency_error->rethrow();
else
LOG_WARNING(log, "{}", inconsistency_error->displayText());
LOG_WARNING(log, getExceptionMessageAndPattern(*inconsistency_error, /* with_stacktrace */ false));
}

auto sleep_time = getSleepTimeAfterInconsistencyError(pass);
Expand Down
2 changes: 1 addition & 1 deletion src/Backups/RestorerFromBackup.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode)

void RestorerFromBackup::setStage(const String & new_stage, const String & message)
{
LOG_TRACE(log, "{}", toUpperFirst(new_stage));
LOG_TRACE(log, fmt::runtime(toUpperFirst(new_stage)));
current_stage = new_stage;

if (restore_coordination)
Expand Down
5 changes: 2 additions & 3 deletions src/Client/ConnectionEstablisher.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -58,9 +58,8 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::
auto table_status_it = status_response.table_states_by_id.find(*table_to_check);
if (table_status_it == status_response.table_states_by_id.end())
{
fail_message = fmt::format("There is no table {}.{} on server: {}",
backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
LOG_WARNING(log, fmt::runtime(fail_message));
LOG_WARNING(LogToStr(fail_message, log), "There is no table {}.{} on server: {}",
backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
return;
}
Expand Down
47 changes: 33 additions & 14 deletions src/Common/Exception.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -71,13 +71,27 @@ Exception::MessageMasked::MessageMasked(const std::string & msg_)
masker->wipeSensitiveData(msg);
}

Exception::MessageMasked::MessageMasked(std::string && msg_)
: msg(std::move(msg_))
{
if (auto * masker = SensitiveDataMasker::getInstance())
masker->wipeSensitiveData(msg);
}

Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code)
, remote(remote_)
{
handle_error_code(msg_masked.msg, code, remote, getStackFramePointers());
}

Exception::Exception(MessageMasked && msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code)
, remote(remote_)
{
handle_error_code(message(), code, remote, getStackFramePointers());
}

Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)
: Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION)
{
Expand Down Expand Up @@ -172,10 +186,11 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string
{
try
{
if (start_of_message.empty())
LOG_ERROR(logger, "{}", getCurrentExceptionMessage(true));
else
LOG_ERROR(logger, "{}: {}", start_of_message, getCurrentExceptionMessage(true));
PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
if (!start_of_message.empty())
message.message = fmt::format("{}: {}", start_of_message, message.message);

LOG_ERROR(logger, message);
}
catch (...)
{
Expand Down Expand Up @@ -323,8 +338,14 @@ std::string getExtraExceptionInfo(const std::exception & e)
}

std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
{
return getCurrentExceptionMessageAndPattern(with_stacktrace, check_embedded_stacktrace, with_extra_info).message;
}

PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
{
WriteBufferFromOwnString stream;
std::string_view message_format_string;

try
{
Expand All @@ -335,6 +356,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
stream << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace)
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
message_format_string = e.tryGetMessageFormatString();
}
catch (const Poco::Exception & e)
{
Expand Down Expand Up @@ -380,7 +402,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
catch (...) {}
}

return stream.str();
return PreformattedMessage{stream.str(), message_format_string};
}


Expand Down Expand Up @@ -433,14 +455,6 @@ int getExceptionErrorCode(std::exception_ptr e)
}


void rethrowFirstException(const Exceptions & exceptions)
{
for (const auto & exception : exceptions)
if (exception)
std::rethrow_exception(exception);
}


void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message)
{
try
Expand All @@ -466,6 +480,11 @@ void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::str
}

std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
{
return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).message;
}

PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
{
WriteBufferFromOwnString stream;

Expand Down Expand Up @@ -497,7 +516,7 @@ std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool
}
catch (...) {}

return stream.str();
return PreformattedMessage{stream.str(), e.tryGetMessageFormatString()};
}

std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace)
Expand Down
61 changes: 51 additions & 10 deletions src/Common/Exception.h
Original file line number Diff line number Diff line change
Expand Up @@ -9,12 +9,32 @@

#include <base/defines.h>
#include <Common/StackTrace.h>
#include <Common/LoggingFormatStringHelpers.h>

#include <fmt/format.h>


namespace Poco { class Logger; }

/// Extract format string from a string literal and constructs consteval fmt::format_string
template <typename... Args>
struct FormatStringHelperImpl
{
std::string_view message_format_string;
fmt::format_string<Args...> fmt_str;
template<typename T>
consteval FormatStringHelperImpl(T && str) : message_format_string(tryGetStaticFormatString(str)), fmt_str(std::forward<T>(str)) {}
template<typename T>
FormatStringHelperImpl(fmt::basic_runtime<T> && str) : message_format_string(), fmt_str(std::forward<fmt::basic_runtime<T>>(str)) {}

PreformattedMessage format(Args && ...args) const
{
return PreformattedMessage{fmt::format(fmt_str, std::forward<Args...>(args)...), message_format_string};
}
};

template <typename... Args>
using FormatStringHelper = FormatStringHelperImpl<std::type_identity_t<Args>...>;

namespace DB
{
Expand All @@ -33,22 +53,37 @@ class Exception : public Poco::Exception
{
std::string msg;
MessageMasked(const std::string & msg_);
MessageMasked(std::string && msg_);
};

Exception(const MessageMasked & msg_masked, int code, bool remote_);
Exception(MessageMasked && msg_masked, int code, bool remote_);

// delegating constructor to mask sensitive information from the message
Exception(const std::string & msg, int code, bool remote_ = false): Exception(MessageMasked(msg), code, remote_)
{}
Exception(const std::string & msg, int code, bool remote_ = false): Exception(MessageMasked(msg), code, remote_) {}
Exception(std::string && msg, int code, bool remote_ = false): Exception(MessageMasked(std::move(msg)), code, remote_) {}
Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.message), code)
{
message_format_string = msg.format_string;
}

Exception(int code, const std::string & message)
template<typename T, typename = std::enable_if_t<std::is_convertible_v<T, String>>>
Exception(int code, T && message)
: Exception(message, code)
{}
{
message_format_string = tryGetStaticFormatString(message);
}

template<> Exception(int code, const String & message) : Exception(message, code) {}
template<> Exception(int code, String & message) : Exception(message, code) {}
template<> Exception(int code, String && message) : Exception(std::move(message), code) {}

// Format message with fmt::format, like the logging functions.
template <typename... Args>
Exception(int code, fmt::format_string<Args...> fmt, Args &&... args) : Exception(fmt::format(fmt, std::forward<Args>(args)...), code)
Exception(int code, FormatStringHelper<Args...> fmt, Args &&... args)
: Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code)
{
message_format_string = fmt.message_format_string;
}

struct CreateFromPocoTag {};
Expand Down Expand Up @@ -87,13 +122,18 @@ class Exception : public Poco::Exception
/// Used for system.errors
FramePointers getStackFramePointers() const;

std::string_view tryGetMessageFormatString() const { return message_format_string; }

private:
#ifndef STD_EXCEPTION_HAS_STACK_TRACE
StackTrace trace;
#endif
bool remote = false;

const char * className() const noexcept override { return "DB::Exception"; }

protected:
std::string_view message_format_string;
};


Expand Down Expand Up @@ -131,14 +171,15 @@ class ParsingException : public Exception
ParsingException();
ParsingException(const std::string & msg, int code);
ParsingException(int code, const std::string & message);
ParsingException(int code, std::string && message) : Exception(message, code) {}

// Format message with fmt::format, like the logging functions.
template <typename... Args>
ParsingException(int code, fmt::format_string<Args...> fmt, Args &&... args) : Exception(code, fmt, std::forward<Args>(args)...)
ParsingException(int code, FormatStringHelper<Args...> fmt, Args &&... args) : Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code)
{
message_format_string = fmt.message_format_string;
}


std::string displayText() const override;

ssize_t getLineNumber() const { return line_number; }
Expand Down Expand Up @@ -184,6 +225,8 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
*/
std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace = false,
bool with_extra_info = true);
PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, bool check_embedded_stacktrace = false,
bool with_extra_info = true);

/// Returns error code from ErrorCodes
int getCurrentExceptionCode();
Expand Down Expand Up @@ -219,12 +262,10 @@ void tryLogException(std::exception_ptr e, const char * log_name, const std::str
void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message = "");

std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false);
PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false);
std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace);


void rethrowFirstException(const Exceptions & exceptions);


template <typename T>
requires std::is_pointer_v<T>
T exception_cast(std::exception_ptr e)
Expand Down
55 changes: 55 additions & 0 deletions src/Common/LoggingFormatStringHelpers.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
#pragma once
#include <base/defines.h>
#include <fmt/format.h>

/// Saves a format string for already formatted message
struct PreformattedMessage
{
String message;
std::string_view format_string;

operator const String & () const { return message; }
operator String () && { return std::move(message); }
operator fmt::format_string<> () const { UNREACHABLE(); }
};

template<typename T> struct is_fmt_runtime : std::false_type {};
template<typename T> struct is_fmt_runtime<fmt::basic_runtime<T>> : std::true_type {};

template <typename T> constexpr std::string_view tryGetStaticFormatString(T && x)
{
/// Failure of this asserting indicates that something went wrong during type deduction.
/// For example, a string literal was implicitly converted to std::string. It should not happen.
static_assert(!std::is_same_v<std::string, std::decay_t<T>>);

if constexpr (is_fmt_runtime<std::decay_t<T>>::value)
{
/// It definitely was fmt::runtime(something).
/// We are not sure about a lifetime of the string, so return empty view.
/// Also it can be arbitrary string, not a formatting pattern.
/// So returning empty pattern will not pollute the set of patterns.
return std::string_view();
}
else
{
if constexpr (std::is_same_v<PreformattedMessage, std::decay_t<T>>)
{
return x.format_string;
}
else
{
/// Most likely it was a string literal.
/// Unfortunately, there's no good way to check if something is a string literal.
/// But fmtlib requires a format string to be compile-time constant unless fmt::runtime is used.
static_assert(std::is_nothrow_convertible<T, const char * const>::value);
static_assert(!std::is_pointer<T>::value);
return std::string_view(x);
}
}
}

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
template <typename T, typename... Ts> constexpr auto firstArg(fmt::basic_runtime<T> && data, Ts &&...) { return data.str.data(); }

2 changes: 1 addition & 1 deletion src/Common/TaskStatsInfoGetter.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ bool checkPermissionsImpl()
{
/// This error happens all the time when running inside Docker - consider it ok,
/// don't create noise with this error.
LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "{}", getCurrentExceptionMessage(false));
LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false));
}
else
{
Expand Down
Loading

0 comments on commit 910d6dc

Please sign in to comment.