Merge pull request #45342 from ClickHouse/exception_message_patterns

Save message format strings for DB::Exception
This commit is contained in:
Alexander Tokmakov 2023-01-20 18:46:52 +03:00 committed by GitHub
commit 910d6dc0ce
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
67 changed files with 379 additions and 303 deletions

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit 0ab9bba7ccad3c8dacce04a35cb3b78218547ab4 Subproject commit 4b1c8dd9913d2a16db62df0e509fa598da5c8219

View File

@ -128,9 +128,10 @@ function run_tests()
if [[ "${HIGH_LEVEL_COVERAGE}" = "YES" ]]; then if [[ "${HIGH_LEVEL_COVERAGE}" = "YES" ]]; then
ADDITIONAL_OPTIONS+=('--report-coverage') ADDITIONAL_OPTIONS+=('--report-coverage')
ADDITIONAL_OPTIONS+=('--report-logs-stats')
fi fi
ADDITIONAL_OPTIONS+=('--report-logs-stats')
set +e set +e
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \

View File

@ -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, /// 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. /// 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);
} }
} }

View File

@ -118,8 +118,7 @@ public:
const auto * y_arg = arguments.at(1).get(); const auto * y_arg = arguments.at(1).get();
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber()) if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.", throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal types of arguments of aggregate function {}, must have number representation.", getName());
ErrorCodes::BAD_ARGUMENTS);
} }
bool allocatesMemoryInArena() const override { return false; } bool allocatesMemoryInArena() const override { return false; }

View File

@ -226,7 +226,7 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{ {
if (!this->data(place).size_x || !this->data(place).size_y) 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); auto [u_statistic, p_value] = this->data(place).getResult(alternative, continuity_correction);

View File

@ -130,7 +130,7 @@ BackupEntries BackupEntriesCollector::run()
Strings BackupEntriesCollector::setStage(const String & new_stage, const String & message) 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; current_stage = new_stage;
backup_coordination->setStage(backup_settings.host_id, new_stage, message); backup_coordination->setStage(backup_settings.host_id, new_stage, message);
@ -215,7 +215,7 @@ void BackupEntriesCollector::gatherMetadataAndCheckConsistency()
if (std::chrono::steady_clock::now() > consistent_metadata_snapshot_end_time) if (std::chrono::steady_clock::now() > consistent_metadata_snapshot_end_time)
inconsistency_error->rethrow(); inconsistency_error->rethrow();
else else
LOG_WARNING(log, "{}", inconsistency_error->displayText()); LOG_WARNING(log, getExceptionMessageAndPattern(*inconsistency_error, /* with_stacktrace */ false));
} }
auto sleep_time = getSleepTimeAfterInconsistencyError(pass); auto sleep_time = getSleepTimeAfterInconsistencyError(pass);

View File

@ -145,7 +145,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode)
void RestorerFromBackup::setStage(const String & new_stage, const String & message) 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; current_stage = new_stage;
if (restore_coordination) if (restore_coordination)

View File

@ -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); 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()) if (table_status_it == status_response.table_states_by_id.end())
{ {
fail_message = fmt::format("There is no table {}.{} on server: {}", 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()); backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
LOG_WARNING(log, fmt::runtime(fail_message));
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
return; return;
} }

View File

@ -71,6 +71,13 @@ Exception::MessageMasked::MessageMasked(const std::string & msg_)
masker->wipeSensitiveData(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_) Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code) : Poco::Exception(msg_masked.msg, code)
, remote(remote_) , remote(remote_)
@ -78,6 +85,13 @@ Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
handle_error_code(msg_masked.msg, code, remote, getStackFramePointers()); 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) Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)
: Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION)
{ {
@ -172,10 +186,11 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string
{ {
try try
{ {
if (start_of_message.empty()) PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
LOG_ERROR(logger, "{}", getCurrentExceptionMessage(true)); if (!start_of_message.empty())
else message.message = fmt::format("{}: {}", start_of_message, message.message);
LOG_ERROR(logger, "{}: {}", start_of_message, getCurrentExceptionMessage(true));
LOG_ERROR(logger, message);
} }
catch (...) catch (...)
{ {
@ -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*/) 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; WriteBufferFromOwnString stream;
std::string_view message_format_string;
try try
{ {
@ -335,6 +356,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
stream << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace) stream << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace)
<< (with_extra_info ? getExtraExceptionInfo(e) : "") << (with_extra_info ? getExtraExceptionInfo(e) : "")
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")"; << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
message_format_string = e.tryGetMessageFormatString();
} }
catch (const Poco::Exception & e) catch (const Poco::Exception & e)
{ {
@ -380,7 +402,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
catch (...) {} catch (...) {}
} }
return stream.str(); return PreformattedMessage{stream.str(), message_format_string};
} }
@ -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) void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message)
{ {
try try
@ -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) 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; WriteBufferFromOwnString stream;
@ -497,7 +516,7 @@ std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool
} }
catch (...) {} catch (...) {}
return stream.str(); return PreformattedMessage{stream.str(), e.tryGetMessageFormatString()};
} }
std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace)

View File

@ -9,12 +9,32 @@
#include <base/defines.h> #include <base/defines.h>
#include <Common/StackTrace.h> #include <Common/StackTrace.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <fmt/format.h> #include <fmt/format.h>
namespace Poco { class Logger; } 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 namespace DB
{ {
@ -33,22 +53,37 @@ public:
{ {
std::string msg; std::string msg;
MessageMasked(const std::string & msg_); MessageMasked(const std::string & msg_);
MessageMasked(std::string && msg_);
}; };
Exception(const MessageMasked & msg_masked, int code, bool remote_); 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 // 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) : 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. // Format message with fmt::format, like the logging functions.
template <typename... Args> 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 {}; struct CreateFromPocoTag {};
@ -87,6 +122,8 @@ public:
/// Used for system.errors /// Used for system.errors
FramePointers getStackFramePointers() const; FramePointers getStackFramePointers() const;
std::string_view tryGetMessageFormatString() const { return message_format_string; }
private: private:
#ifndef STD_EXCEPTION_HAS_STACK_TRACE #ifndef STD_EXCEPTION_HAS_STACK_TRACE
StackTrace trace; StackTrace trace;
@ -94,6 +131,9 @@ private:
bool remote = false; bool remote = false;
const char * className() const noexcept override { return "DB::Exception"; } const char * className() const noexcept override { return "DB::Exception"; }
protected:
std::string_view message_format_string;
}; };
@ -131,14 +171,15 @@ public:
ParsingException(); ParsingException();
ParsingException(const std::string & msg, int code); ParsingException(const std::string & msg, int code);
ParsingException(int code, const std::string & message); 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. // Format message with fmt::format, like the logging functions.
template <typename... Args> 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; std::string displayText() const override;
ssize_t getLineNumber() const { return line_number; } ssize_t getLineNumber() const { return line_number; }
@ -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, std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace = false,
bool with_extra_info = true); 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 /// Returns error code from ErrorCodes
int getCurrentExceptionCode(); int getCurrentExceptionCode();
@ -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 = ""); 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); 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); std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace);
void rethrowFirstException(const Exceptions & exceptions);
template <typename T> template <typename T>
requires std::is_pointer_v<T> requires std::is_pointer_v<T>
T exception_cast(std::exception_ptr e) T exception_cast(std::exception_ptr e)

View File

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

View File

@ -212,7 +212,7 @@ bool checkPermissionsImpl()
{ {
/// This error happens all the time when running inside Docker - consider it ok, /// This error happens all the time when running inside Docker - consider it ok,
/// don't create noise with this error. /// 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 else
{ {

View File

@ -6,6 +6,9 @@
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Poco/Message.h> #include <Poco/Message.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Common/LoggingFormatStringHelpers.h>
namespace Poco { class Logger; }
/// This wrapper is useful to save formatted message into a String before sending it to a logger /// This wrapper is useful to save formatted message into a String before sending it to a logger
class LogToStrImpl class LogToStrImpl
@ -33,40 +36,9 @@ public:
namespace namespace
{ {
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(); }
[[maybe_unused]] const ::Poco::Logger * getLogger(const ::Poco::Logger * logger) { return logger; }; [[maybe_unused]] const ::Poco::Logger * getLogger(const ::Poco::Logger * logger) { return logger; };
[[maybe_unused]] const ::Poco::Logger * getLogger(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); }; [[maybe_unused]] const ::Poco::Logger * getLogger(const std::atomic<::Poco::Logger *> & logger) { return logger.load(); };
[[maybe_unused]] std::unique_ptr<LogToStrImpl> getLogger(std::unique_ptr<LogToStrImpl> && logger) { return logger; }; [[maybe_unused]] std::unique_ptr<LogToStrImpl> getLogger(std::unique_ptr<LogToStrImpl> && logger) { return logger; };
template<typename T> struct is_fmt_runtime : std::false_type {};
template<typename T> struct is_fmt_runtime<fmt::basic_runtime<T>> : std::true_type {};
/// Usually we use LOG_*(...) macros with either string literals or fmt::runtime(whatever) as a format string.
/// This function is useful to get a string_view to a static format string passed to LOG_* macro.
template <typename T> constexpr std::string_view tryGetStaticFormatString(T && x)
{
if constexpr (is_fmt_runtime<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
{
/// 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);
}
}
} }
#define LOG_IMPL_FIRST_ARG(X, ...) X #define LOG_IMPL_FIRST_ARG(X, ...) X

View File

@ -357,16 +357,12 @@ nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
if (read_checksum != hash.get64()) if (read_checksum != hash.get64())
{ {
const auto error_string = fmt::format( constexpr auto error_format = "Invalid checksum while reading state from {}. Got {}, expected {}";
"Invalid checksum while reading state from {}. Got {}, expected {}",
path.generic_string(),
hash.get64(),
read_checksum);
#ifdef NDEBUG #ifdef NDEBUG
LOG_ERROR(logger, fmt::runtime(error_string)); LOG_ERROR(logger, error_format, path.generic_string(), hash.get64(), read_checksum);
return nullptr; return nullptr;
#else #else
throw Exception(ErrorCodes::CORRUPTED_DATA, error_string); throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, path.generic_string(), hash.get64(), read_checksum);
#endif #endif
} }

View File

@ -278,7 +278,7 @@ private:
if (next_pos != std::string_view::npos) if (next_pos != std::string_view::npos)
size = next_pos - pos; size = next_pos - pos;
LOG_FATAL(log, "{}", message.substr(pos, size)); LOG_FATAL(log, fmt::runtime(message.substr(pos, size)));
pos = next_pos; pos = next_pos;
} }
} }

View File

@ -82,7 +82,7 @@ void DatabaseAtomic::drop(ContextPtr)
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
fs::remove_all(getMetadataPath()); fs::remove_all(getMetadataPath());
} }
@ -477,7 +477,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String &
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
} }
@ -490,7 +490,7 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name)
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
} }
@ -535,7 +535,7 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(true))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
} }
auto new_name_escaped = escapeForFileName(new_name); auto new_name_escaped = escapeForFileName(new_name);

View File

@ -100,7 +100,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, Co
if (!load_result.config) if (!load_result.config)
{ {
if (throw_on_error) if (throw_on_error)
throw Exception{"Dictionary " + backQuote(table_name) + " doesn't exist", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY}; throw Exception(ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY, "Dictionary {} doesn't exist", backQuote(table_name));
return {}; return {};
} }

View File

@ -102,7 +102,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name
auto it = tables_cache.find(table_name); auto it = tables_cache.find(table_name);
if (it != tables_cache.end()) if (it != tables_cache.end())
return it->second.metadata_modification_time; return it->second.metadata_modification_time;
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", backQuote(database_name), backQuote(table_name));
} }
void DatabaseLazy::alterTable( void DatabaseLazy::alterTable(
@ -185,7 +185,7 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
auto it = tables_cache.find(table_name); auto it = tables_cache.find(table_name);
if (it == tables_cache.end()) if (it == tables_cache.end())
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.", backQuote(database_name), backQuote(table_name));
res = it->second.table; res = it->second.table;
if (it->second.expiration_iterator != cache_expiration_queue.end()) if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator); cache_expiration_queue.erase(it->second.expiration_iterator);

View File

@ -312,7 +312,7 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na
} }
catch (...) catch (...)
{ {
LOG_WARNING(log, fmt::runtime(getCurrentExceptionMessage(__PRETTY_FUNCTION__))); LOG_WARNING(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true));
attachTable(local_context, table_name, table, table_data_path_relative); attachTable(local_context, table_name, table, table_data_path_relative);
if (renamed) if (renamed)
fs::rename(table_metadata_path_drop, table_metadata_path); fs::rename(table_metadata_path_drop, table_metadata_path);
@ -463,8 +463,7 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, Contex
catch (const Exception & e) catch (const Exception & e)
{ {
if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error) if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error)
throw Exception{"Table " + backQuote(table_name) + " doesn't exist", throw Exception(ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, "Table {} doesn't exist", backQuote(table_name));
ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY};
else if (!is_system_storage && throw_on_error) else if (!is_system_storage && throw_on_error)
throw; throw;
} }

View File

@ -136,8 +136,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context
if (local_tables_cache.find(table_name) == local_tables_cache.end()) if (local_tables_cache.find(table_name) == local_tables_cache.end())
{ {
if (throw_on_error) if (throw_on_error)
throw Exception("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist..", throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
ErrorCodes::UNKNOWN_TABLE);
return nullptr; return nullptr;
} }
@ -181,7 +180,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam
fetchTablesIntoLocalCache(getContext()); fetchTablesIntoLocalCache(getContext());
if (local_tables_cache.find(table_name) == local_tables_cache.end()) if (local_tables_cache.find(table_name) == local_tables_cache.end())
throw Exception("MySQL table " + database_name_in_mysql + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.", database_name_in_mysql, table_name);
return time_t(local_tables_cache[table_name].first); return time_t(local_tables_cache[table_name].first);
} }

View File

@ -113,8 +113,8 @@ DiskSelectorPtr DiskSelector::updateFromConfig(
writeBackQuotedString(name, warning); writeBackQuotedString(name, warning);
} }
writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); LOG_WARNING(&Poco::Logger::get("DiskSelector"), "{} disappeared from configuration, "
LOG_WARNING(&Poco::Logger::get("DiskSelector"), fmt::runtime(warning.str())); "this change will be applied after restart of ClickHouse", warning.str());
} }
return result; return result;

View File

@ -26,7 +26,7 @@ inline void throwIfDivisionLeadsToFPE(A a, B b)
/// Is it better to use siglongjmp instead of checks? /// Is it better to use siglongjmp instead of checks?
if (unlikely(b == 0)) if (unlikely(b == 0))
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION); throw Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
/// http://avva.livejournal.com/2548306.html /// http://avva.livejournal.com/2548306.html
if (unlikely(is_signed_v<A> && is_signed_v<B> && a == std::numeric_limits<A>::min() && b == -1)) if (unlikely(is_signed_v<A> && is_signed_v<B> && a == std::numeric_limits<A>::min() && b == -1))

View File

@ -184,15 +184,11 @@ void validateFunctionArgumentTypes(const IFunction & func,
return result; return result;
}; };
throw Exception("Incorrect number of arguments for function " + func.getName() throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
+ " provided " + std::to_string(arguments.size()) "Incorrect number of arguments for function {} provided {}{}, expected {}{} ({}{})",
+ (!arguments.empty() ? " (" + join_argument_types(arguments) + ")" : String{}) func.getName(), arguments.size(), (!arguments.empty() ? " (" + join_argument_types(arguments) + ")" : String{}),
+ ", expected " + std::to_string(mandatory_args.size()) mandatory_args.size(), (!optional_args.empty() ? " to " + std::to_string(mandatory_args.size() + optional_args.size()) : ""),
+ (!optional_args.empty() ? " to " + std::to_string(mandatory_args.size() + optional_args.size()) : "") join_argument_types(mandatory_args), (!optional_args.empty() ? ", [" + join_argument_types(optional_args) + "]" : ""));
+ " (" + join_argument_types(mandatory_args)
+ (!optional_args.empty() ? ", [" + join_argument_types(optional_args) + "]" : "")
+ ")",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
} }
validateArgumentsImpl(func, arguments, 0, mandatory_args); validateArgumentsImpl(func, arguments, 0, mandatory_args);

View File

@ -1166,8 +1166,8 @@ public:
} }
catch (const Exception &) catch (const Exception &)
{ {
throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")" throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments ({}, {})"
" of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); " of function {}", arguments[0]->getName(), arguments[1]->getName(), getName());
} }
} }

View File

@ -1116,7 +1116,7 @@ inline bool tryParseImpl<DataTypeIPv6>(DataTypeIPv6::FieldType & x, ReadBuffer &
if (isNativeNumber(result_type) && !(result_type.getName() == "IPv4" || result_type.getName() == "IPv6")) 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."; 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(message_buf.str(), ErrorCodes::CANNOT_PARSE_TEXT); throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}"}, ErrorCodes::CANNOT_PARSE_TEXT);
} }

View File

@ -7,7 +7,7 @@ namespace DB
REGISTER_FUNCTION(ExternalDictionaries) REGISTER_FUNCTION(ExternalDictionaries)
{ {
const std::string dict_get_description { R"( constexpr auto dict_get_description { R"(
Retrieves values from a dictionary. Retrieves values from a dictionary.
Accepts 3 parameters: Accepts 3 parameters:
@ -20,7 +20,7 @@ Returned value: value of the dictionary attribute parsed in the {} if key is fou
Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type. Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type.
)" }; )" };
const std::string dict_get_or_default_description { R"( constexpr auto dict_get_or_default_description { R"(
Retrieves values from a dictionary. Retrieves values from a dictionary.
Accepts 4 parameters: Accepts 4 parameters:
@ -34,7 +34,7 @@ Returned value: value of the dictionary attribute parsed in the {} if key is fou
Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type. Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type.
)" }; )" };
const std::string dict_get_or_null_description { R"( constexpr auto dict_get_or_null_description { R"(
Retrieves values from a dictionary. Retrieves values from a dictionary.
Accepts 3 parameters: Accepts 3 parameters:
@ -47,43 +47,43 @@ Returned value: value of the dictionary attribute parsed in the attributes da
Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type. Throws an exception if cannot parse the value of the attribute or the value does not match the attribute data type.
)" }; )" };
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "attributes data type") }); factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>(Documentation{ fmt::format(dict_get_description, "attributes data type") });
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "attributes data type") }); factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>(Documentation{ fmt::format(dict_get_or_default_description, "attributes data type") });
factory.registerFunction<FunctionDictGetOrNull>(Documentation{ dict_get_or_null_description }); factory.registerFunction<FunctionDictGetOrNull>(Documentation{ dict_get_or_null_description });
factory.registerFunction<FunctionDictGetUInt8>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt8") }); factory.registerFunction<FunctionDictGetUInt8>(Documentation{ fmt::format(dict_get_description, "UInt8") });
factory.registerFunction<FunctionDictGetUInt16>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt16") }); factory.registerFunction<FunctionDictGetUInt16>(Documentation{ fmt::format(dict_get_description, "UInt16") });
factory.registerFunction<FunctionDictGetUInt32>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt32") }); factory.registerFunction<FunctionDictGetUInt32>(Documentation{ fmt::format(dict_get_description, "UInt32") });
factory.registerFunction<FunctionDictGetUInt64>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UInt64") }); factory.registerFunction<FunctionDictGetUInt64>(Documentation{ fmt::format(dict_get_description, "UInt64") });
factory.registerFunction<FunctionDictGetInt8>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int8") }); factory.registerFunction<FunctionDictGetInt8>(Documentation{ fmt::format(dict_get_description, "Int8") });
factory.registerFunction<FunctionDictGetInt16>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int16") }); factory.registerFunction<FunctionDictGetInt16>(Documentation{ fmt::format(dict_get_description, "Int16") });
factory.registerFunction<FunctionDictGetInt32>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int32") }); factory.registerFunction<FunctionDictGetInt32>(Documentation{ fmt::format(dict_get_description, "Int32") });
factory.registerFunction<FunctionDictGetInt64>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Int64") }); factory.registerFunction<FunctionDictGetInt64>(Documentation{ fmt::format(dict_get_description, "Int64") });
factory.registerFunction<FunctionDictGetFloat32>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Float32") }); factory.registerFunction<FunctionDictGetFloat32>(Documentation{ fmt::format(dict_get_description, "Float32") });
factory.registerFunction<FunctionDictGetFloat64>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Float64") }); factory.registerFunction<FunctionDictGetFloat64>(Documentation{ fmt::format(dict_get_description, "Float64") });
factory.registerFunction<FunctionDictGetDate>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "Date") }); factory.registerFunction<FunctionDictGetDate>(Documentation{ fmt::format(dict_get_description, "Date") });
factory.registerFunction<FunctionDictGetDateTime>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "DateTime") }); factory.registerFunction<FunctionDictGetDateTime>(Documentation{ fmt::format(dict_get_description, "DateTime") });
factory.registerFunction<FunctionDictGetUUID>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "UUID") }); factory.registerFunction<FunctionDictGetUUID>(Documentation{ fmt::format(dict_get_description, "UUID") });
factory.registerFunction<FunctionDictGetIPv4>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "IPv4") }); factory.registerFunction<FunctionDictGetIPv4>(Documentation{ fmt::format(dict_get_description, "IPv4") });
factory.registerFunction<FunctionDictGetIPv6>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "IPv6") }); factory.registerFunction<FunctionDictGetIPv6>(Documentation{ fmt::format(dict_get_description, "IPv6") });
factory.registerFunction<FunctionDictGetString>(Documentation{ fmt::format(fmt::runtime(dict_get_description), "String") }); factory.registerFunction<FunctionDictGetString>(Documentation{ fmt::format(dict_get_description, "String") });
factory.registerFunction<FunctionDictGetUInt8OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt8") }); factory.registerFunction<FunctionDictGetUInt8OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "UInt8") });
factory.registerFunction<FunctionDictGetUInt16OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt16") }); factory.registerFunction<FunctionDictGetUInt16OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "UInt16") });
factory.registerFunction<FunctionDictGetUInt32OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt32") }); factory.registerFunction<FunctionDictGetUInt32OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "UInt32") });
factory.registerFunction<FunctionDictGetUInt64OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UInt64") }); factory.registerFunction<FunctionDictGetUInt64OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "UInt64") });
factory.registerFunction<FunctionDictGetInt8OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int8") }); factory.registerFunction<FunctionDictGetInt8OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "Int8") });
factory.registerFunction<FunctionDictGetInt16OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int16") }); factory.registerFunction<FunctionDictGetInt16OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "Int16") });
factory.registerFunction<FunctionDictGetInt32OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int32") }); factory.registerFunction<FunctionDictGetInt32OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "Int32") });
factory.registerFunction<FunctionDictGetInt64OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Int64") }); factory.registerFunction<FunctionDictGetInt64OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "Int64") });
factory.registerFunction<FunctionDictGetFloat32OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Float32") }); factory.registerFunction<FunctionDictGetFloat32OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "Float32") });
factory.registerFunction<FunctionDictGetFloat64OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Float64") }); factory.registerFunction<FunctionDictGetFloat64OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "Float64") });
factory.registerFunction<FunctionDictGetDateOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "Date") }); factory.registerFunction<FunctionDictGetDateOrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "Date") });
factory.registerFunction<FunctionDictGetDateTimeOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "DateTime") }); factory.registerFunction<FunctionDictGetDateTimeOrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "DateTime") });
factory.registerFunction<FunctionDictGetUUIDOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "UUID") }); factory.registerFunction<FunctionDictGetUUIDOrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "UUID") });
factory.registerFunction<FunctionDictGetIPv4OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "IPv4") }); factory.registerFunction<FunctionDictGetIPv4OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "IPv4") });
factory.registerFunction<FunctionDictGetIPv6OrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "IPv6") }); factory.registerFunction<FunctionDictGetIPv6OrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "IPv6") });
factory.registerFunction<FunctionDictGetStringOrDefault>(Documentation{ fmt::format(fmt::runtime(dict_get_or_default_description), "String") }); factory.registerFunction<FunctionDictGetStringOrDefault>(Documentation{ fmt::format(dict_get_or_default_description, "String") });
factory.registerFunction<FunctionDictHas>(Documentation{ R"( factory.registerFunction<FunctionDictHas>(Documentation{ R"(
Checks whether a key is present in a dictionary. Checks whether a key is present in a dictionary.

View File

@ -207,7 +207,7 @@ public:
if (arguments.size() < 2) if (arguments.size() < 2)
{ {
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "{}", arguments.size()); throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect number of arguments: {}", arguments.size());
} }
const auto * first_array_type = checkAndGetDataType<typename Impl::data_type>(arguments[1].type.get()); const auto * first_array_type = checkAndGetDataType<typename Impl::data_type>(arguments[1].type.get());

View File

@ -22,7 +22,7 @@ struct DivideDecimalsImpl
execute(FirstType a, SecondType b, UInt16 scale_a, UInt16 scale_b, UInt16 result_scale) execute(FirstType a, SecondType b, UInt16 scale_a, UInt16 scale_b, UInt16 result_scale)
{ {
if (b.value == 0) if (b.value == 0)
throw DB::Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION); throw DB::Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
if (a.value == 0) if (a.value == 0)
return Decimal256(0); return Decimal256(0);

View File

@ -78,7 +78,7 @@ struct DivideIntegralByConstantImpl
#pragma GCC diagnostic pop #pragma GCC diagnostic pop
if (unlikely(static_cast<A>(b) == 0)) if (unlikely(static_cast<A>(b) == 0))
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION); throw Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
divideImpl(a_pos, b, c_pos, size); divideImpl(a_pos, b, c_pos, size);
} }

View File

@ -78,7 +78,7 @@ struct ModuloByConstantImpl
#pragma GCC diagnostic pop #pragma GCC diagnostic pop
if (unlikely(static_cast<A>(b) == 0)) if (unlikely(static_cast<A>(b) == 0))
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION); throw Exception(ErrorCodes::ILLEGAL_DIVISION, "Division by zero");
/// Division by min negative value. /// Division by min negative value.
if (std::is_signed_v<B> && b == std::numeric_limits<B>::lowest()) if (std::is_signed_v<B> && b == std::numeric_limits<B>::lowest())

View File

@ -61,20 +61,17 @@ public:
size_t number_of_arguments = arguments.size(); size_t number_of_arguments = arguments.size();
if (number_of_arguments < 2 || number_of_arguments > 3) if (number_of_arguments < 2 || number_of_arguments > 3)
throw Exception( throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function " + getName() + " doesn't match: passed " + toString(number_of_arguments) "Number of arguments for function {} doesn't match: passed {}, should be from 2 to 3",
+ ", should be from 2 to 3", getName(), toString(number_of_arguments));
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
// second argument must be an integer // second argument must be an integer
if (!isInteger(arguments[1])) if (!isInteger(arguments[1]))
throw Exception( throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - should be an integer", "Illegal type {} of second argument of function {} - should be an integer", arguments[1]->getName(), getName());
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else if (arguments[1]->isNullable()) else if (arguments[1]->isNullable())
throw Exception( throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + " - can not be Nullable", "Illegal type {} of second argument of function {} - can not be Nullable", arguments[1]->getName(), getName());
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
// check that default value column has supertype with first argument // check that default value column has supertype with first argument
if (number_of_arguments == 3) if (number_of_arguments == 3)

View File

@ -58,24 +58,20 @@ public:
size_t number_of_arguments = arguments.size(); size_t number_of_arguments = arguments.size();
if (number_of_arguments < 2 || number_of_arguments > 3) if (number_of_arguments < 2 || number_of_arguments > 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: "
+ toString(number_of_arguments) + ", should be 2 or 3", "passed {}, should be 2 or 3", getName(), number_of_arguments);
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if ((is_utf8 && !isString(arguments[0])) || !isStringOrFixedString(arguments[0])) if ((is_utf8 && !isString(arguments[0])) || !isStringOrFixedString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}",
arguments[0]->getName(), getName());
if (!isNativeNumber(arguments[1])) if (!isNativeNumber(arguments[1]))
throw Exception("Illegal type " + arguments[1]->getName() throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}",
+ " of second argument of function " arguments[1]->getName(), getName());
+ getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (number_of_arguments == 3 && !isNativeNumber(arguments[2])) if (number_of_arguments == 3 && !isNativeNumber(arguments[2]))
throw Exception("Illegal type " + arguments[2]->getName() throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}",
+ " of second argument of function " arguments[2]->getName(), getName());
+ getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
} }
@ -99,7 +95,7 @@ public:
sliceFromRightConstantOffsetUnbounded( sliceFromRightConstantOffsetUnbounded(
source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(start_value)); source, StringSink(*col_res, input_rows_count), -static_cast<size_t>(start_value));
else else
throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); throw Exception(ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX, "Indices in strings are 1-based");
} }
else else
sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start); sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start);

View File

@ -324,13 +324,14 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR
} }
} }
std::string HTTPException::makeExceptionMessage( Exception HTTPException::makeExceptionMessage(
int code,
const std::string & uri, const std::string & uri,
Poco::Net::HTTPResponse::HTTPStatus http_status, Poco::Net::HTTPResponse::HTTPStatus http_status,
const std::string & reason, const std::string & reason,
const std::string & body) const std::string & body)
{ {
return fmt::format( return Exception(code,
"Received error from remote server {}. " "Received error from remote server {}. "
"HTTP status code: {} {}, " "HTTP status code: {} {}, "
"body: {}", "body: {}",

View File

@ -43,7 +43,7 @@ public:
const std::string & reason, const std::string & reason,
const std::string & body const std::string & body
) )
: Exception(makeExceptionMessage(uri, http_status_, reason, body), code) : Exception(makeExceptionMessage(code, uri, http_status_, reason, body))
, http_status(http_status_) , http_status(http_status_)
{} {}
@ -55,7 +55,8 @@ public:
private: private:
Poco::Net::HTTPResponse::HTTPStatus http_status{}; Poco::Net::HTTPResponse::HTTPStatus http_status{};
static std::string makeExceptionMessage( static Exception makeExceptionMessage(
int code,
const std::string & uri, const std::string & uri,
Poco::Net::HTTPResponse::HTTPStatus http_status, Poco::Net::HTTPResponse::HTTPStatus http_status,
const std::string & reason, const std::string & reason,

View File

@ -132,7 +132,7 @@ public:
const auto & [level, prio] = convertLogLevel(log_level); const auto & [level, prio] = convertLogLevel(log_level);
if (tag_loggers.contains(tag)) if (tag_loggers.contains(tag))
{ {
LOG_IMPL(tag_loggers[tag], level, prio, "{}", message); LOG_IMPL(tag_loggers[tag], level, prio, fmt::runtime(message));
} }
else else
{ {

View File

@ -1083,7 +1083,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
size_t num_result_columns = result.size(); size_t num_result_columns = result.size();
if (mode == MatchColumnsMode::Position && num_input_columns != num_result_columns) if (mode == MatchColumnsMode::Position && num_input_columns != num_result_columns)
throw Exception("Number of columns doesn't match", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns doesn't match");
if (add_casted_columns && mode != MatchColumnsMode::Name) if (add_casted_columns && mode != MatchColumnsMode::Name)
throw Exception("Converting with add_casted_columns supported only for MatchColumnsMode::Name", ErrorCodes::LOGICAL_ERROR); throw Exception("Converting with add_casted_columns supported only for MatchColumnsMode::Name", ErrorCodes::LOGICAL_ERROR);

View File

@ -268,7 +268,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
{ {
assert(!db_and_table.first && !db_and_table.second); assert(!db_and_table.first && !db_and_table.second);
if (exception) if (exception)
exception->emplace(fmt::format("Table {} doesn't exist", table_id.getNameForLogs()), ErrorCodes::UNKNOWN_TABLE); exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
return {}; return {};
} }
@ -317,7 +317,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
auto table = database->tryGetTable(table_id.table_name, context_); auto table = database->tryGetTable(table_id.table_name, context_);
if (!table && exception) if (!table && exception)
exception->emplace(fmt::format("Table {} doesn't exist", table_id.getNameForLogs()), ErrorCodes::UNKNOWN_TABLE); exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
if (!table) if (!table)
database = nullptr; database = nullptr;
@ -382,7 +382,7 @@ void DatabaseCatalog::assertDatabaseExistsUnlocked(const String & database_name)
{ {
assert(!database_name.empty()); assert(!database_name.empty());
if (databases.end() == databases.find(database_name)) if (databases.end() == databases.find(database_name))
throw Exception("Database " + backQuoteIfNeed(database_name) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exist", backQuoteIfNeed(database_name));
} }
@ -544,7 +544,7 @@ bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id, ContextPtr co
void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id, ContextPtr context_) const void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id, ContextPtr context_) const
{ {
if (isTableExist(table_id, context_)) if (isTableExist(table_id, context_))
throw Exception("Table " + table_id.getNameForLogs() + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id);
} }
DatabasePtr DatabaseCatalog::getDatabaseForTemporaryTables() const DatabasePtr DatabaseCatalog::getDatabaseForTemporaryTables() const

View File

@ -111,8 +111,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
{ {
if (query.if_exists) if (query.if_exists)
return {}; return {};
throw Exception("Temporary table " + backQuoteIfNeed(table_id.table_name) + " doesn't exist", throw Exception(ErrorCodes::UNKNOWN_TABLE, "Temporary table {} doesn't exist", backQuoteIfNeed(table_id.table_name));
ErrorCodes::UNKNOWN_TABLE);
} }
auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr);

View File

@ -61,8 +61,7 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline()
storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext());
if (!storage) if (!storage)
throw Exception("Table " + table_id.getNameForLogs() + " doesn't exist.", throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist.", table_id.getNameForLogs());
ErrorCodes::UNKNOWN_TABLE);
auto storage_name = storage->getName(); auto storage_name = storage->getName();
if (storage_name == "LiveView" if (storage_name == "LiveView"

View File

@ -163,8 +163,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
if (running_query != user_process_list->second.queries.end()) if (running_query != user_process_list->second.queries.end())
{ {
if (!settings.replace_running_query) if (!settings.replace_running_query)
throw Exception("Query with id = " + client_info.current_query_id + " is already running.", throw Exception(ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING, "Query with id = {} is already running.", client_info.current_query_id);
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
/// Ask queries to cancel. They will check this flag. /// Ask queries to cancel. They will check this flag.
running_query->second->is_killed.store(true, std::memory_order_relaxed); running_query->second->is_killed.store(true, std::memory_order_relaxed);
@ -180,8 +179,9 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
return false; return false;
})) }))
{ {
throw Exception("Query with id = " + client_info.current_query_id + " is already running and can't be stopped", throw Exception(ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING,
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING); "Query with id = {} is already running and can't be stopped",
client_info.current_query_id);
} }
} }
} }
@ -193,8 +193,9 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
if (user_process_list.first == client_info.current_user) if (user_process_list.first == client_info.current_user)
continue; continue;
if (auto running_query = user_process_list.second.queries.find(client_info.current_query_id); running_query != user_process_list.second.queries.end()) if (auto running_query = user_process_list.second.queries.find(client_info.current_query_id); running_query != user_process_list.second.queries.end())
throw Exception("Query with id = " + client_info.current_query_id + " is already running by user " + user_process_list.first, throw Exception(ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING,
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING); "Query with id = {} is already running by user {}",
client_info.current_query_id, user_process_list.first);
} }
auto user_process_list_it = user_to_queries.find(client_info.current_user); auto user_process_list_it = user_to_queries.find(client_info.current_user);
@ -397,7 +398,7 @@ void QueryStatus::addPipelineExecutor(PipelineExecutor * e)
/// addPipelineExecutor() from the cancelQuery() context, and this will /// addPipelineExecutor() from the cancelQuery() context, and this will
/// lead to deadlock. /// lead to deadlock.
if (is_killed.load()) if (is_killed.load())
throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED); throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled");
std::lock_guard lock(executors_mutex); std::lock_guard lock(executors_mutex);
assert(std::find(executors.begin(), executors.end(), e) == executors.end()); assert(std::find(executors.begin(), executors.end(), e) == executors.end());
@ -414,7 +415,7 @@ void QueryStatus::removePipelineExecutor(PipelineExecutor * e)
bool QueryStatus::checkTimeLimit() bool QueryStatus::checkTimeLimit()
{ {
if (is_killed.load()) if (is_killed.load())
throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED); throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled");
return limits.checkTimeLimit(watch, overflow_mode); return limits.checkTimeLimit(watch, overflow_mode);
} }

View File

@ -79,6 +79,7 @@ struct QueryLogElement
Int32 exception_code{}; // because ErrorCodes are int Int32 exception_code{}; // because ErrorCodes are int
String exception; String exception;
String stack_trace; String stack_trace;
std::string_view exception_format_string{};
ClientInfo client_info; ClientInfo client_info;

View File

@ -181,24 +181,26 @@ static void logException(ContextPtr context, QueryLogElement & elem)
if (!elem.log_comment.empty()) if (!elem.log_comment.empty())
comment = fmt::format(" (comment: {})", elem.log_comment); comment = fmt::format(" (comment: {})", elem.log_comment);
/// Message patterns like "{} (from {}){} (in query: {})" are not really informative,
/// so we pass elem.exception_format_string as format string instead.
PreformattedMessage message;
message.format_string = elem.exception_format_string;
if (elem.stack_trace.empty()) if (elem.stack_trace.empty())
LOG_ERROR( message.message = fmt::format("{} (from {}){} (in query: {})", elem.exception,
&Poco::Logger::get("executeQuery"), context->getClientInfo().current_address.toString(),
"{} (from {}){} (in query: {})", comment,
elem.exception, toOneLineQuery(elem.query));
context->getClientInfo().current_address.toString(),
comment,
toOneLineQuery(elem.query));
else else
LOG_ERROR( message.message = fmt::format(
&Poco::Logger::get("executeQuery"), "{} (from {}){} (in query: {}), Stack trace (when copying this message, always include the lines below):\n\n{}",
"{} (from {}){} (in query: {})"
", Stack trace (when copying this message, always include the lines below):\n\n{}",
elem.exception, elem.exception,
context->getClientInfo().current_address.toString(), context->getClientInfo().current_address.toString(),
comment, comment,
toOneLineQuery(elem.query), toOneLineQuery(elem.query),
elem.stack_trace); elem.stack_trace);
LOG_ERROR(&Poco::Logger::get("executeQuery"), message);
} }
static void onExceptionBeforeStart( static void onExceptionBeforeStart(
@ -243,7 +245,9 @@ static void onExceptionBeforeStart(
// We don't calculate databases, tables and columns when the query isn't able to start // We don't calculate databases, tables and columns when the query isn't able to start
elem.exception_code = getCurrentExceptionCode(); elem.exception_code = getCurrentExceptionCode();
elem.exception = getCurrentExceptionMessage(false); auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false);
elem.exception = std::move(exception_message.message);
elem.exception_format_string = exception_message.format_string;
elem.client_info = context->getClientInfo(); elem.client_info = context->getClientInfo();
@ -1019,7 +1023,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING;
elem.exception_code = getCurrentExceptionCode(); elem.exception_code = getCurrentExceptionCode();
elem.exception = getCurrentExceptionMessage(false); auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false);
elem.exception = std::move(exception_message.message);
elem.exception_format_string = exception_message.format_string;
QueryStatusPtr process_list_elem = context->getProcessListElement(); QueryStatusPtr process_list_elem = context->getProcessListElement();
const Settings & current_settings = context->getSettingsRef(); const Settings & current_settings = context->getSettingsRef();

View File

@ -101,7 +101,7 @@ void ExecutionSpeedLimits::throttle(
} }
template <typename... Args> template <typename... Args>
static bool handleOverflowMode(OverflowMode mode, int code, fmt::format_string<Args...> fmt, Args &&... args) static bool handleOverflowMode(OverflowMode mode, int code, FormatStringHelper<Args...> fmt, Args &&... args)
{ {
switch (mode) switch (mode)
{ {

View File

@ -93,7 +93,7 @@ void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & conf
catch (...) catch (...)
{ {
init_was_not_made = true; init_was_not_made = true;
LOG_ERROR(log, fmt::runtime(getCurrentExceptionMessage(false))); LOG_ERROR(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false));
} }
} }
} }

View File

@ -1296,7 +1296,7 @@ namespace
{ {
io.onException(); io.onException();
LOG_ERROR(log, fmt::runtime(getExceptionMessage(exception, true))); LOG_ERROR(log, getExceptionMessageAndPattern(exception, /* with_stacktrace */ true));
if (responder && !responder_finished) if (responder && !responder_finished)
{ {

View File

@ -134,21 +134,21 @@ void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPSe
/// Sending to remote server was cancelled due to server shutdown or drop table. /// Sending to remote server was cancelled due to server shutdown or drop table.
bool is_real_error = e.code() != ErrorCodes::ABORTED; bool is_real_error = e.code() != ErrorCodes::ABORTED;
std::string message = getCurrentExceptionMessage(is_real_error); PreformattedMessage message = getCurrentExceptionMessageAndPattern(is_real_error);
write_response(message); write_response(message.message);
if (is_real_error) if (is_real_error)
LOG_ERROR(log, fmt::runtime(message)); LOG_ERROR(log, message);
else else
LOG_INFO(log, fmt::runtime(message)); LOG_INFO(log, message);
} }
catch (...) catch (...)
{ {
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
std::string message = getCurrentExceptionMessage(false); PreformattedMessage message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false);
write_response(message); write_response(message.message);
LOG_ERROR(log, fmt::runtime(message)); LOG_ERROR(log, message);
} }
} }

View File

@ -537,7 +537,7 @@ void TCPHandler::runImpl()
} }
const auto & e = *exception; const auto & e = *exception;
LOG_ERROR(log, fmt::runtime(getExceptionMessage(e, true))); LOG_ERROR(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ true));
sendException(*exception, send_exception_with_stack_trace); sendException(*exception, send_exception_with_stack_trace);
} }
} }

View File

@ -1065,19 +1065,20 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
if (all_columns.has(column_name) || all_columns.hasNested(column_name)) if (all_columns.has(column_name) || all_columns.hasNested(column_name))
{ {
if (!command.if_not_exists) if (!command.if_not_exists)
throw Exception{"Cannot add column " + backQuote(column_name) + ": column with this name already exists", throw Exception(ErrorCodes::DUPLICATE_COLUMN,
ErrorCodes::DUPLICATE_COLUMN}; "Cannot add column {}: column with this name already exists",
backQuote(column_name));
else else
continue; continue;
} }
if (!command.data_type) if (!command.data_type)
throw Exception{"Data type have to be specified for column " + backQuote(column_name) + " to add", throw Exception(ErrorCodes::BAD_ARGUMENTS,
ErrorCodes::BAD_ARGUMENTS}; "Data type have to be specified for column {} to add", backQuote(column_name));
if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table)) if (column_name == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table))
throw Exception{"Cannot add column " + backQuote(column_name) + ": this column name is reserved for lightweight delete feature", throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot add column {}: "
ErrorCodes::ILLEGAL_COLUMN}; "this column name is reserved for lightweight delete feature", backQuote(column_name));
if (command.codec) if (command.codec)
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs);
@ -1100,8 +1101,8 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
} }
if (renamed_columns.contains(column_name)) if (renamed_columns.contains(column_name))
throw Exception{"Cannot rename and modify the same column " + backQuote(column_name) + " in a single ALTER query", throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rename and modify the same column {} "
ErrorCodes::NOT_IMPLEMENTED}; "in a single ALTER query", backQuote(column_name));
if (command.codec) if (command.codec)
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs); CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context->getSettingsRef().allow_suspicious_codecs, context->getSettingsRef().allow_experimental_codecs);
@ -1260,16 +1261,16 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
} }
if (all_columns.has(command.rename_to)) if (all_columns.has(command.rename_to))
throw Exception{"Cannot rename to " + backQuote(command.rename_to) + ": column with this name already exists", throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Cannot rename to {}: "
ErrorCodes::DUPLICATE_COLUMN}; "column with this name already exists", backQuote(command.rename_to));
if (command.rename_to == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table)) if (command.rename_to == LightweightDeleteDescription::FILTER_COLUMN.name && std::dynamic_pointer_cast<MergeTreeData>(table))
throw Exception{"Cannot rename to " + backQuote(command.rename_to) + ": this column name is reserved for lightweight delete feature", throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot rename to {}: "
ErrorCodes::ILLEGAL_COLUMN}; "this column name is reserved for lightweight delete feature", backQuote(command.rename_to));
if (modified_columns.contains(column_name)) if (modified_columns.contains(column_name))
throw Exception{"Cannot rename and modify the same column " + backQuote(column_name) + " in a single ALTER query", throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot rename and modify the same column {} "
ErrorCodes::NOT_IMPLEMENTED}; "in a single ALTER query", backQuote(column_name));
String from_nested_table_name = Nested::extractTableName(command.column_name); String from_nested_table_name = Nested::extractTableName(command.column_name);
String to_nested_table_name = Nested::extractTableName(command.rename_to); String to_nested_table_name = Nested::extractTableName(command.rename_to);
@ -1279,7 +1280,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
if (from_nested && to_nested) if (from_nested && to_nested)
{ {
if (from_nested_table_name != to_nested_table_name) if (from_nested_table_name != to_nested_table_name)
throw Exception{"Cannot rename column from one nested name to another", ErrorCodes::BAD_ARGUMENTS}; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot rename column from one nested name to another");
} }
else if (!from_nested && !to_nested) else if (!from_nested && !to_nested)
{ {
@ -1289,16 +1290,16 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
} }
else else
{ {
throw Exception{"Cannot rename column from nested struct to normal column and vice versa", ErrorCodes::BAD_ARGUMENTS}; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot rename column from nested struct to normal column and vice versa");
} }
} }
else if (command.type == AlterCommand::REMOVE_TTL && !metadata.hasAnyTableTTL()) else if (command.type == AlterCommand::REMOVE_TTL && !metadata.hasAnyTableTTL())
{ {
throw Exception{"Table doesn't have any table TTL expression, cannot remove", ErrorCodes::BAD_ARGUMENTS}; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table doesn't have any table TTL expression, cannot remove");
} }
else if (command.type == AlterCommand::REMOVE_SAMPLE_BY && !metadata.hasSamplingKey()) else if (command.type == AlterCommand::REMOVE_SAMPLE_BY && !metadata.hasSamplingKey())
{ {
throw Exception{"Table doesn't have SAMPLE BY, cannot remove", ErrorCodes::BAD_ARGUMENTS}; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table doesn't have SAMPLE BY, cannot remove");
} }
/// Collect default expressions for MODIFY and ADD commands /// Collect default expressions for MODIFY and ADD commands
@ -1342,7 +1343,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
} }
if (all_columns.empty()) if (all_columns.empty())
throw Exception{"Cannot DROP or CLEAR all columns", ErrorCodes::BAD_ARGUMENTS}; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot DROP or CLEAR all columns");
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context); validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
} }

View File

@ -200,8 +200,8 @@ static auto getNameRange(const ColumnsDescription::ColumnsContainer & columns, c
void ColumnsDescription::add(ColumnDescription column, const String & after_column, bool first, bool add_subcolumns) void ColumnsDescription::add(ColumnDescription column, const String & after_column, bool first, bool add_subcolumns)
{ {
if (has(column.name)) if (has(column.name))
throw Exception("Cannot add column " + column.name + ": column with this name already exists", throw Exception(ErrorCodes::ILLEGAL_COLUMN,
ErrorCodes::ILLEGAL_COLUMN); "Cannot add column {}: column with this name already exists", column.name);
/// Normalize ASTs to be compatible with InterpreterCreateQuery. /// Normalize ASTs to be compatible with InterpreterCreateQuery.
if (column.default_desc.expression) if (column.default_desc.expression)

View File

@ -445,7 +445,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart(
DiskPtr disk) DiskPtr disk)
{ {
if (blocker.isCancelled()) if (blocker.isCancelled())
throw Exception("Fetching of part was cancelled", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Fetching of part was cancelled");
const auto data_settings = data.getSettings(); const auto data_settings = data.getSettings();
@ -794,7 +794,7 @@ void Fetcher::downloadBasePartOrProjectionPartToDiskRemoteMeta(
/// NOTE The is_cancelled flag also makes sense to check every time you read over the network, /// NOTE The is_cancelled flag also makes sense to check every time you read over the network,
/// performing a poll with a not very large timeout. /// performing a poll with a not very large timeout.
/// And now we check it only between read chunks (in the `copyData` function). /// And now we check it only between read chunks (in the `copyData` function).
throw Exception("Fetching of part was cancelled", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Fetching of part was cancelled");
} }
MergeTreeDataPartChecksum::uint128 expected_hash; MergeTreeDataPartChecksum::uint128 expected_hash;
@ -853,7 +853,7 @@ void Fetcher::downloadBaseOrProjectionPartToDisk(
/// NOTE The is_cancelled flag also makes sense to check every time you read over the network, /// NOTE The is_cancelled flag also makes sense to check every time you read over the network,
/// performing a poll with a not very large timeout. /// performing a poll with a not very large timeout.
/// And now we check it only between read chunks (in the `copyData` function). /// And now we check it only between read chunks (in the `copyData` function).
throw Exception("Fetching of part was cancelled", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Fetching of part was cancelled");
} }
MergeTreeDataPartChecksum::uint128 expected_hash; MergeTreeDataPartChecksum::uint128 expected_hash;

View File

@ -106,12 +106,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
const String local_tmp_suffix = global_ctx->parent_part ? ctx->suffix : ""; const String local_tmp_suffix = global_ctx->parent_part ? ctx->suffix : "";
if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed)) if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed))
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts");
/// We don't want to perform merge assigned with TTL as normal merge, so /// We don't want to perform merge assigned with TTL as normal merge, so
/// throw exception /// throw exception
if (isTTLMergeType(global_ctx->future_part->merge_type) && global_ctx->ttl_merges_blocker->isCancelled()) if (isTTLMergeType(global_ctx->future_part->merge_type) && global_ctx->ttl_merges_blocker->isCancelled())
throw Exception("Cancelled merging parts with TTL", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts with TTL");
LOG_DEBUG(ctx->log, "Merging {} parts: from {} to {} into {}", LOG_DEBUG(ctx->log, "Merging {} parts: from {} to {} into {}",
global_ctx->future_part->parts.size(), global_ctx->future_part->parts.size(),
@ -412,10 +412,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl()
global_ctx->merged_pipeline.reset(); global_ctx->merged_pipeline.reset();
if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed)) if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed))
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts");
if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled()) if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled())
throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts with expired TTL");
const auto data_settings = global_ctx->data->getSettings(); const auto data_settings = global_ctx->data->getSettings();
const size_t sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; const size_t sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed;
@ -541,7 +541,7 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const
{ {
const String & column_name = ctx->it_name_and_type->name; const String & column_name = ctx->it_name_and_type->name;
if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed)) if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed))
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts");
ctx->executor.reset(); ctx->executor.reset();
auto changed_checksums = ctx->column_to->fillChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns); auto changed_checksums = ctx->column_to->fillChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns);

View File

@ -140,7 +140,7 @@ void MergeTreeBackgroundExecutor<Queue>::routine(TaskRuntimeDataPtr item)
NOEXCEPT_SCOPE({ NOEXCEPT_SCOPE({
ALLOW_ALLOCATIONS_IN_SCOPE; ALLOW_ALLOCATIONS_IN_SCOPE;
if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info.
LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
else else
tryLogCurrentException(__PRETTY_FUNCTION__); tryLogCurrentException(__PRETTY_FUNCTION__);
}); });
@ -200,7 +200,7 @@ void MergeTreeBackgroundExecutor<Queue>::routine(TaskRuntimeDataPtr item)
NOEXCEPT_SCOPE({ NOEXCEPT_SCOPE({
ALLOW_ALLOCATIONS_IN_SCOPE; ALLOW_ALLOCATIONS_IN_SCOPE;
if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info. if (e.code() == ErrorCodes::ABORTED) /// Cancelled merging parts is not an error - log as info.
LOG_INFO(log, fmt::runtime(getCurrentExceptionMessage(false))); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
else else
tryLogCurrentException(__PRETTY_FUNCTION__); tryLogCurrentException(__PRETTY_FUNCTION__);
}); });

View File

@ -2869,15 +2869,13 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
} }
else if (command.type == AlterCommand::DROP_COLUMN) else if (command.type == AlterCommand::DROP_COLUMN)
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER DROP version " + backQuoteIfNeed(command.column_name) + " column", "Trying to ALTER DROP version {} column", backQuoteIfNeed(command.column_name));
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
} }
else if (command.type == AlterCommand::RENAME_COLUMN) else if (command.type == AlterCommand::RENAME_COLUMN)
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER RENAME version " + backQuoteIfNeed(command.column_name) + " column", "Trying to ALTER RENAME version {} column", backQuoteIfNeed(command.column_name));
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
} }
} }
@ -2929,9 +2927,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
{ {
if (columns_in_keys.contains(command.column_name)) if (columns_in_keys.contains(command.column_name))
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER DROP key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression", "Trying to ALTER DROP key {} column which is a part of key expression", backQuoteIfNeed(command.column_name));
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
} }
if (!command.clear) if (!command.clear)
@ -2939,10 +2936,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];
if (!deps_mv.empty()) if (!deps_mv.empty())
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view " "Trying to ALTER DROP column {} which is referenced by materialized view {}",
+ toString(deps_mv), backQuoteIfNeed(command.column_name), toString(deps_mv));
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
} }
} }

View File

@ -387,7 +387,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn(
const auto & tuple_data_type = typeid_cast<const DataTypeTuple *>(type.get()); const auto & tuple_data_type = typeid_cast<const DataTypeTuple *>(type.get());
if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size) if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size)
throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name);
bool match_with_subtype = false; bool match_with_subtype = false;
const auto & sub_columns = tuple_column->getColumns(); const auto & sub_columns = tuple_column->getColumns();
@ -678,7 +678,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals(
const auto * value_tuple_data_type = typeid_cast<const DataTypeTuple *>(value_type.get()); const auto * value_tuple_data_type = typeid_cast<const DataTypeTuple *>(value_type.get());
if (tuple.size() != key_node_function_arguments_size) if (tuple.size() != key_node_function_arguments_size)
throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types of arguments of function {}", function_name);
bool match_with_subtype = false; bool match_with_subtype = false;
const DataTypes & subtypes = value_tuple_data_type->getElements(); const DataTypes & subtypes = value_tuple_data_type->getElements();

View File

@ -98,9 +98,9 @@ bool MutatePlainMergeTreeTask::executeStep()
{ {
if (merge_mutate_entry->txn) if (merge_mutate_entry->txn)
merge_mutate_entry->txn->onException(); merge_mutate_entry->txn->onException();
String exception_message = getCurrentExceptionMessage(false); PreformattedMessage exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false);
LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), "{}", exception_message); LOG_ERROR(&Poco::Logger::get("MutatePlainMergeTreeTask"), exception_message);
storage.updateMutationEntriesErrors(future_part, false, exception_message); storage.updateMutationEntriesErrors(future_part, false, exception_message.message);
write_part_log(ExecutionStatus::fromCurrentException()); write_part_log(ExecutionStatus::fromCurrentException());
tryLogCurrentException(__PRETTY_FUNCTION__); tryLogCurrentException(__PRETTY_FUNCTION__);
return false; return false;

View File

@ -42,7 +42,7 @@ namespace MutationHelpers
static bool checkOperationIsNotCanceled(ActionBlocker & merges_blocker, MergeListEntry * mutate_entry) static bool checkOperationIsNotCanceled(ActionBlocker & merges_blocker, MergeListEntry * mutate_entry)
{ {
if (merges_blocker.isCancelled() || (*mutate_entry)->is_cancelled) if (merges_blocker.isCancelled() || (*mutate_entry)->is_cancelled)
throw Exception("Cancelled mutating parts", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Cancelled mutating parts");
return true; return true;
} }

View File

@ -46,19 +46,19 @@ bool ReplicatedMergeMutateTaskBase::executeStep()
if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART) if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
{ {
/// If no one has the right part, probably not all replicas work; We will not write to log with Error level. /// If no one has the right part, probably not all replicas work; We will not write to log with Error level.
LOG_INFO(log, fmt::runtime(e.displayText())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
retryable_error = true; retryable_error = true;
} }
else if (e.code() == ErrorCodes::ABORTED) else if (e.code() == ErrorCodes::ABORTED)
{ {
/// Interrupted merge or downloading a part is not an error. /// Interrupted merge or downloading a part is not an error.
LOG_INFO(log, fmt::runtime(e.message())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
retryable_error = true; retryable_error = true;
} }
else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED) else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
{ {
/// Part cannot be added temporarily /// Part cannot be added temporarily
LOG_INFO(log, fmt::runtime(e.displayText())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
retryable_error = true; retryable_error = true;
storage.cleanup_thread.wakeup(); storage.cleanup_thread.wakeup();
} }

View File

@ -332,8 +332,8 @@ size_t ReplicatedMergeTreeSinkImpl<async_insert>::checkQuorumPrecondition(const
String quorum_status; String quorum_status;
if (!quorum_parallel && zookeeper->tryGet(quorum_info.status_path, quorum_status)) if (!quorum_parallel && zookeeper->tryGet(quorum_info.status_path, quorum_status))
throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status, throw Exception(ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE,
ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); "Quorum for previous write has not been satisfied yet. Status: {}", quorum_status);
/// Both checks are implicitly made also later (otherwise there would be a race condition). /// Both checks are implicitly made also later (otherwise there would be a race condition).

View File

@ -1026,10 +1026,9 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];
if (!deps_mv.empty()) if (!deps_mv.empty())
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view " "Trying to ALTER DROP column {} which is referenced by materialized view {}",
+ toString(deps_mv), backQuoteIfNeed(command.column_name), toString(deps_mv));
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
} }
} }
} }

View File

@ -1065,10 +1065,9 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, Co
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];
if (!deps_mv.empty()) if (!deps_mv.empty())
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view " "Trying to ALTER DROP column {} which is referenced by materialized view {}",
+ toString(deps_mv), backQuoteIfNeed(command.column_name), toString(deps_mv));
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
} }
} }
} }

View File

@ -77,26 +77,26 @@ public:
auto is_table_valid = isTableValid(); auto is_table_valid = isTableValid();
if (!is_table_valid.has_value()) if (!is_table_valid.has_value())
{ {
static constexpr std::string_view error_msg = "Failed to activate table because of connection issues. It will be activated " static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated "
"once a connection is established and metadata is verified"; "once a connection is established and metadata is verified";
if constexpr (throw_on_error) if constexpr (throw_on_error)
throw Exception(ErrorCodes::INVALID_STATE, error_msg); throw Exception(ErrorCodes::INVALID_STATE, error_msg);
else else
{ {
LOG_ERROR(log, fmt::runtime(error_msg)); LOG_ERROR(log, error_msg);
return; return;
} }
} }
if (!*is_table_valid) if (!*is_table_valid)
{ {
static constexpr std::string_view error_msg static constexpr auto error_msg
= "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table"; = "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table";
if constexpr (throw_on_error) if constexpr (throw_on_error)
throw Exception(ErrorCodes::INVALID_STATE, error_msg); throw Exception(ErrorCodes::INVALID_STATE, error_msg);
else else
{ {
LOG_ERROR(log, fmt::runtime(error_msg)); LOG_ERROR(log, error_msg);
return; return;
} }
} }

View File

@ -849,10 +849,9 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextP
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];
if (!deps_mv.empty()) if (!deps_mv.empty())
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view " "Trying to ALTER DROP column {} which is referenced by materialized view {}",
+ toString(deps_mv), backQuoteIfNeed(command.column_name), toString(deps_mv));
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
} }
} }
} }

View File

@ -922,7 +922,7 @@ bool StorageMergeTree::merge(
{ {
std::unique_lock lock(currently_processing_in_background_mutex); std::unique_lock lock(currently_processing_in_background_mutex);
if (merger_mutator.merges_blocker.isCancelled()) if (merger_mutator.merges_blocker.isCancelled())
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts");
merge_mutate_entry = selectPartsToMerge( merge_mutate_entry = selectPartsToMerge(
metadata_snapshot, metadata_snapshot,

View File

@ -54,10 +54,10 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];
if (!deps_mv.empty()) if (!deps_mv.empty())
{ {
throw Exception( throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view " "Trying to ALTER DROP column {} which is referenced by materialized view {}",
+ toString(deps_mv), backQuoteIfNeed(command.column_name), toString(deps_mv)
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); );
} }
} }
} }

View File

@ -2929,7 +2929,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke
} }
if (source_replica.empty()) if (source_replica.empty())
throw Exception("All replicas are lost", ErrorCodes::ALL_REPLICAS_LOST); throw Exception(ErrorCodes::ALL_REPLICAS_LOST, "All replicas are lost");
if (is_new_replica) if (is_new_replica)
LOG_INFO(log, "Will mimic {}", source_replica); LOG_INFO(log, "Will mimic {}", source_replica);
@ -3049,17 +3049,17 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel
if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART) if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
{ {
/// If no one has the right part, probably not all replicas work; We will not write to log with Error level. /// If no one has the right part, probably not all replicas work; We will not write to log with Error level.
LOG_INFO(log, fmt::runtime(e.displayText())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
} }
else if (e.code() == ErrorCodes::ABORTED) else if (e.code() == ErrorCodes::ABORTED)
{ {
/// Interrupted merge or downloading a part is not an error. /// Interrupted merge or downloading a part is not an error.
LOG_INFO(log, fmt::runtime(e.message())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
} }
else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED) else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
{ {
/// Part cannot be added temporarily /// Part cannot be added temporarily
LOG_INFO(log, fmt::runtime(e.displayText())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
cleanup_thread.wakeup(); cleanup_thread.wakeup();
} }
else else
@ -4787,13 +4787,14 @@ bool StorageReplicatedMergeTree::optimize(
assertNotReadonly(); assertNotReadonly();
if (!is_leader) if (!is_leader)
throw Exception("OPTIMIZE cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER); throw Exception(ErrorCodes::NOT_A_LEADER, "OPTIMIZE cannot be done on this replica because it is not a leader");
auto handle_noop = [&] (const char * fmt_string, auto ...args) auto handle_noop = [&]<typename... Args>(FormatStringHelper<Args...> fmt_string, Args && ...args)
{ {
LOG_DEBUG(log, fmt::runtime(fmt_string), args...); PreformattedMessage message = fmt_string.format(std::forward<Args...>(args)...);
LOG_DEBUG(log, message);
if (query_context->getSettingsRef().optimize_throw_if_noop) if (query_context->getSettingsRef().optimize_throw_if_noop)
throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, fmt::runtime(fmt_string), args...); throw Exception(message, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
return false; return false;
}; };
@ -6116,7 +6117,7 @@ void StorageReplicatedMergeTree::fetchPartition(
&& e.code() != ErrorCodes::CANNOT_READ_ALL_DATA) && e.code() != ErrorCodes::CANNOT_READ_ALL_DATA)
throw; throw;
LOG_INFO(log, fmt::runtime(e.displayText())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
} }
return; return;
} }
@ -6252,7 +6253,7 @@ void StorageReplicatedMergeTree::fetchPartition(
&& e.code() != ErrorCodes::CANNOT_READ_ALL_DATA) && e.code() != ErrorCodes::CANNOT_READ_ALL_DATA)
throw; throw;
LOG_INFO(log, fmt::runtime(e.displayText())); LOG_INFO(log, getExceptionMessageAndPattern(e, /* with_stacktrace */ false));
} }
if (!fetched) if (!fetched)

View File

@ -67,8 +67,8 @@ ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr contex
const auto columns = tables_and_columns.find(configuration->table); const auto columns = tables_and_columns.find(configuration->table);
if (columns == tables_and_columns.end()) if (columns == tables_and_columns.end())
throw Exception("MySQL table " + (configuration->database.empty() ? "" : (backQuote(configuration->database) + ".")) throw Exception(ErrorCodes::UNKNOWN_TABLE, "MySQL table {} doesn't exist.",
+ backQuote(configuration->table) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); (configuration->database.empty() ? "" : (backQuote(configuration->database) + "." + backQuote(configuration->table))));
return columns->second; return columns->second;
} }

View File

@ -3,6 +3,7 @@
# pylint: disable=too-many-return-statements # pylint: disable=too-many-return-statements
# pylint: disable=global-variable-not-assigned # pylint: disable=global-variable-not-assigned
# pylint: disable=too-many-lines # pylint: disable=too-many-lines
# pylint: disable=anomalous-backslash-in-string
import enum import enum
from queue import Full from queue import Full
@ -1854,7 +1855,8 @@ def reportLogStats(args):
SELECT SELECT
count() AS count, count() AS count,
substr(replaceRegexpAll(message, '[^A-Za-z]+', ''), 1, 32) AS pattern, substr(replaceRegexpAll(message, '[^A-Za-z]+', ''), 1, 32) AS pattern,
substr(any(message), 1, 256) as runtime_message substr(any(message), 1, 256) as runtime_message,
any((extract(source_file, '\/[a-zA-Z0-9_]+\.[a-z]+'), source_line)) as line
FROM system.text_log FROM system.text_log
WHERE (now() - toIntervalMinute(mins)) < event_time AND message_format_string = '' WHERE (now() - toIntervalMinute(mins)) < event_time AND message_format_string = ''
GROUP BY pattern GROUP BY pattern