ClickHouse/src/Common/Exception.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

620 lines
18 KiB
C++
Raw Normal View History

2019-03-27 15:42:24 +00:00
#include "Exception.h"
#include <cstring>
2014-07-21 09:11:20 +00:00
#include <cxxabi.h>
2020-10-15 18:14:04 +00:00
#include <cstdlib>
2017-02-07 16:36:12 +00:00
#include <Poco/String.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadBufferFromFile.h>
2021-10-02 07:13:14 +00:00
#include <base/demangle.h>
#include <base/errnoToString.h>
2019-08-05 19:41:20 +00:00
#include <Common/formatReadable.h>
2019-11-27 09:39:44 +00:00
#include <Common/filesystemHelpers.h>
#include <Common/ErrorCodes.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/LockMemoryExceptionInThread.h>
2019-08-05 19:41:20 +00:00
#include <filesystem>
2010-03-01 16:59:51 +00:00
#include "config_version.h"
2021-05-13 09:32:52 +00:00
namespace fs = std::filesystem;
2010-03-01 16:59:51 +00:00
namespace DB
{
2011-12-12 06:15:34 +00:00
namespace ErrorCodes
{
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
extern const int LOGICAL_ERROR;
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int CANNOT_MREMAP;
}
2022-05-20 10:41:44 +00:00
void abortOnFailedAssertion(const String & description)
{
LOG_FATAL(&Poco::Logger::root(), "Logical error: '{}'.", description);
/// This is to suppress -Wmissing-noreturn
volatile bool always_false = false;
if (always_false)
return;
abort();
}
/// - Aborts the process if error code is LOGICAL_ERROR.
/// - Increments error codes statistics.
void handle_error_code([[maybe_unused]] const std::string & msg, int code, bool remote, const Exception::FramePointers & trace)
{
2020-10-15 18:14:04 +00:00
// In debug builds and builds with sanitizers, treat LOGICAL_ERROR as an assertion failure.
// Log the message before we fail.
2020-10-16 11:58:47 +00:00
#ifdef ABORT_ON_LOGICAL_ERROR
if (code == ErrorCodes::LOGICAL_ERROR)
{
2022-05-20 10:41:44 +00:00
abortOnFailedAssertion(msg);
}
#endif
ErrorCodes::increment(code, remote, msg, trace);
}
Exception::MessageMasked::MessageMasked(const std::string & msg_)
: msg(msg_)
{
if (auto * masker = SensitiveDataMasker::getInstance())
masker->wipeSensitiveData(msg);
}
2023-01-16 22:20:33 +00:00
Exception::MessageMasked::MessageMasked(std::string && msg_)
: msg(std::move(msg_))
{
if (auto * masker = SensitiveDataMasker::getInstance())
masker->wipeSensitiveData(msg);
}
Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code)
, remote(remote_)
{
handle_error_code(msg_masked.msg, code, remote, getStackFramePointers());
}
2023-01-16 22:20:33 +00:00
Exception::Exception(MessageMasked && msg_masked, int code, bool remote_)
: Poco::Exception(msg_masked.msg, code)
, remote(remote_)
{
handle_error_code(message(), code, remote, getStackFramePointers());
}
Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)
: Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION)
{
2020-01-02 10:29:59 +00:00
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
set_stack_trace(exc.get_stack_trace_frames(), exc.get_stack_trace_size());
2020-01-02 10:29:59 +00:00
#endif
}
Exception::Exception(CreateFromSTDTag, const std::exception & exc)
: Poco::Exception(demangle(typeid(exc).name()) + ": " + String(exc.what()), ErrorCodes::STD_EXCEPTION)
{
2020-01-02 10:29:59 +00:00
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
set_stack_trace(exc.get_stack_trace_frames(), exc.get_stack_trace_size());
2020-01-02 10:29:59 +00:00
#endif
}
std::string getExceptionStackTraceString(const std::exception & e)
{
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
return StackTrace::toString(e.get_stack_trace_frames(), 0, e.get_stack_trace_size());
#else
2020-01-02 18:55:45 +00:00
if (const auto * db_exception = dynamic_cast<const Exception *>(&e))
2020-01-02 16:32:17 +00:00
return db_exception->getStackTraceString();
return {};
#endif
}
2021-06-18 16:25:19 +00:00
std::string getExceptionStackTraceString(std::exception_ptr e)
{
try
{
std::rethrow_exception(e);
}
catch (const std::exception & exception)
{
return getExceptionStackTraceString(exception);
}
catch (...)
{
return {};
}
}
std::string Exception::getStackTraceString() const
{
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
return StackTrace::toString(get_stack_trace_frames(), 0, get_stack_trace_size());
#else
return trace.toString();
#endif
}
Exception::FramePointers Exception::getStackFramePointers() const
{
2021-03-24 06:41:57 +00:00
FramePointers frame_pointers;
#ifdef STD_EXCEPTION_HAS_STACK_TRACE
{
2021-03-24 06:41:57 +00:00
frame_pointers.resize(get_stack_trace_size());
for (size_t i = 0; i < frame_pointers.size(); ++i)
{
2021-03-24 06:41:57 +00:00
frame_pointers[i] = get_stack_trace_frames()[i];
}
}
#else
{
size_t stack_trace_size = trace.getSize();
size_t stack_trace_offset = trace.getOffset();
2021-03-24 06:41:57 +00:00
frame_pointers.reserve(stack_trace_size - stack_trace_offset);
for (size_t i = stack_trace_offset; i < stack_trace_size; ++i)
{
2021-03-24 06:41:57 +00:00
frame_pointers.push_back(trace.getFramePointers()[i]);
}
}
#endif
2021-03-24 06:41:57 +00:00
return frame_pointers;
}
2020-03-08 21:04:10 +00:00
void throwFromErrno(const std::string & s, int code, int the_errno)
{
2022-08-20 15:09:20 +00:00
throw ErrnoException(s + ", " + errnoToString(the_errno), code, the_errno);
}
2019-08-07 12:52:47 +00:00
void throwFromErrnoWithPath(const std::string & s, const std::string & path, int code, int the_errno)
2019-08-06 18:54:06 +00:00
{
2022-08-20 15:09:20 +00:00
throw ErrnoException(s + ", " + errnoToString(the_errno), code, the_errno, path);
2019-08-06 18:54:06 +00:00
}
2021-05-28 14:34:16 +00:00
static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message)
{
try
{
2023-01-16 22:20:33 +00:00
PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
if (!start_of_message.empty())
2023-01-25 20:16:42 +00:00
message.text = fmt::format("{}: {}", start_of_message, message.text);
2023-01-16 22:20:33 +00:00
LOG_ERROR(logger, message);
2021-05-28 14:34:16 +00:00
}
catch (...)
{
}
}
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
{
feat: implement catboost in library-bridge This commit moves the catboost model evaluation out of the server process into the library-bridge binary. This serves two goals: On the one hand, crashes / memory corruptions of the catboost library no longer affect the server. On the other hand, we can forbid loading dynamic libraries in the server (catboost was the last consumer of this functionality), thus improving security. SQL syntax: SELECT catboostEvaluate('/path/to/model.bin', FEAT_1, ..., FEAT_N) > 0 AS prediction, ACTION AS target FROM amazon_train LIMIT 10 Required configuration: <catboost_lib_path>/path/to/libcatboostmodel.so</catboost_lib_path> *** Implementation Details *** The internal protocol between the server and the library-bridge is simple: - HTTP GET on path "/extdict_ping": A ping, used during the handshake to check if the library-bridge runs. - HTTP POST on path "extdict_request" (1) Send a "catboost_GetTreeCount" request from the server to the bridge, containing a library path (e.g /home/user/libcatboost.so) and a model path (e.g. /home/user/model.bin). Rirst, this unloads the catboost library handler associated to the model path (if it was loaded), then loads the catboost library handler associated to the model path, then executes GetTreeCount() on the library handler and finally sends the result back to the server. Step (1) is called once by the server from FunctionCatBoostEvaluate::getReturnTypeImpl(). The library path handler is unloaded in the beginning because it contains state which may no longer be valid if the user runs catboost("/path/to/model.bin", ...) more than once and if "model.bin" was updated in between. (2) Send "catboost_Evaluate" from the server to the bridge, containing the model path and the features to run the interference on. Step (2) is called multiple times (once per chunk) by the server from function FunctionCatBoostEvaluate::executeImpl(). The library handler for the given model path is expected to be already loaded by Step (1). Fixes #27870
2022-08-05 07:53:06 +00:00
/// Under high memory pressure, new allocations throw a
/// MEMORY_LIMIT_EXCEEDED exception.
2021-05-28 14:34:16 +00:00
///
feat: implement catboost in library-bridge This commit moves the catboost model evaluation out of the server process into the library-bridge binary. This serves two goals: On the one hand, crashes / memory corruptions of the catboost library no longer affect the server. On the other hand, we can forbid loading dynamic libraries in the server (catboost was the last consumer of this functionality), thus improving security. SQL syntax: SELECT catboostEvaluate('/path/to/model.bin', FEAT_1, ..., FEAT_N) > 0 AS prediction, ACTION AS target FROM amazon_train LIMIT 10 Required configuration: <catboost_lib_path>/path/to/libcatboostmodel.so</catboost_lib_path> *** Implementation Details *** The internal protocol between the server and the library-bridge is simple: - HTTP GET on path "/extdict_ping": A ping, used during the handshake to check if the library-bridge runs. - HTTP POST on path "extdict_request" (1) Send a "catboost_GetTreeCount" request from the server to the bridge, containing a library path (e.g /home/user/libcatboost.so) and a model path (e.g. /home/user/model.bin). Rirst, this unloads the catboost library handler associated to the model path (if it was loaded), then loads the catboost library handler associated to the model path, then executes GetTreeCount() on the library handler and finally sends the result back to the server. Step (1) is called once by the server from FunctionCatBoostEvaluate::getReturnTypeImpl(). The library path handler is unloaded in the beginning because it contains state which may no longer be valid if the user runs catboost("/path/to/model.bin", ...) more than once and if "model.bin" was updated in between. (2) Send "catboost_Evaluate" from the server to the bridge, containing the model path and the features to run the interference on. Step (2) is called multiple times (once per chunk) by the server from function FunctionCatBoostEvaluate::executeImpl(). The library handler for the given model path is expected to be already loaded by Step (1). Fixes #27870
2022-08-05 07:53:06 +00:00
/// In this case the exception will not be logged, so let's block the
2021-05-28 14:34:16 +00:00
/// MemoryTracker until the exception will be logged.
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
2021-05-28 14:34:16 +00:00
/// Poco::Logger::get can allocate memory too
tryLogCurrentExceptionImpl(&Poco::Logger::get(log_name), start_of_message);
}
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message)
2013-11-18 19:18:03 +00:00
{
feat: implement catboost in library-bridge This commit moves the catboost model evaluation out of the server process into the library-bridge binary. This serves two goals: On the one hand, crashes / memory corruptions of the catboost library no longer affect the server. On the other hand, we can forbid loading dynamic libraries in the server (catboost was the last consumer of this functionality), thus improving security. SQL syntax: SELECT catboostEvaluate('/path/to/model.bin', FEAT_1, ..., FEAT_N) > 0 AS prediction, ACTION AS target FROM amazon_train LIMIT 10 Required configuration: <catboost_lib_path>/path/to/libcatboostmodel.so</catboost_lib_path> *** Implementation Details *** The internal protocol between the server and the library-bridge is simple: - HTTP GET on path "/extdict_ping": A ping, used during the handshake to check if the library-bridge runs. - HTTP POST on path "extdict_request" (1) Send a "catboost_GetTreeCount" request from the server to the bridge, containing a library path (e.g /home/user/libcatboost.so) and a model path (e.g. /home/user/model.bin). Rirst, this unloads the catboost library handler associated to the model path (if it was loaded), then loads the catboost library handler associated to the model path, then executes GetTreeCount() on the library handler and finally sends the result back to the server. Step (1) is called once by the server from FunctionCatBoostEvaluate::getReturnTypeImpl(). The library path handler is unloaded in the beginning because it contains state which may no longer be valid if the user runs catboost("/path/to/model.bin", ...) more than once and if "model.bin" was updated in between. (2) Send "catboost_Evaluate" from the server to the bridge, containing the model path and the features to run the interference on. Step (2) is called multiple times (once per chunk) by the server from function FunctionCatBoostEvaluate::executeImpl(). The library handler for the given model path is expected to be already loaded by Step (1). Fixes #27870
2022-08-05 07:53:06 +00:00
/// Under high memory pressure, new allocations throw a
/// MEMORY_LIMIT_EXCEEDED exception.
///
/// And in this case the exception will not be logged, so let's block the
/// MemoryTracker until the exception will be logged.
LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global);
2021-05-28 14:34:16 +00:00
tryLogCurrentExceptionImpl(logger, start_of_message);
}
2021-05-13 09:32:52 +00:00
static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg)
2019-08-05 19:41:20 +00:00
{
2019-08-06 18:54:06 +00:00
path = std::filesystem::absolute(path);
2019-08-05 19:41:20 +00:00
/// It's possible to get ENOSPC for non existent file (e.g. if there are no free inodes and creat() fails)
/// So try to get info for existent parent directory.
while (!std::filesystem::exists(path) && path.has_relative_path())
path = path.parent_path();
2019-11-27 09:39:44 +00:00
auto fs = getStatVFS(path);
auto mount_point = getMountPoint(path).string();
2020-05-30 21:35:52 +00:00
fmt::format_to(std::back_inserter(msg),
"\nTotal space: {}\nAvailable space: {}\nTotal inodes: {}\nAvailable inodes: {}\nMount point: {}",
ReadableSize(fs.f_blocks * fs.f_frsize),
ReadableSize(fs.f_bavail * fs.f_frsize),
2020-05-30 21:35:52 +00:00
formatReadableQuantity(fs.f_files),
formatReadableQuantity(fs.f_favail),
mount_point);
#if defined(OS_LINUX)
2019-11-27 09:39:44 +00:00
msg += "\nFilesystem: " + getFilesystemName(mount_point);
2019-08-07 12:52:47 +00:00
#endif
2019-08-05 19:41:20 +00:00
}
/** It is possible that the system has enough memory,
* but we have shortage of the number of available memory mappings.
* Provide good diagnostic to user in that case.
*/
static void getNotEnoughMemoryMessage(std::string & msg)
{
#if defined(OS_LINUX)
try
{
2020-06-08 17:35:45 +00:00
static constexpr size_t buf_size = 1024;
2020-05-30 20:02:44 +00:00
char buf[buf_size];
UInt64 max_map_count = 0;
{
2020-05-30 20:02:44 +00:00
ReadBufferFromFile file("/proc/sys/vm/max_map_count", buf_size, -1, buf);
readText(max_map_count, file);
}
UInt64 num_maps = 0;
{
2020-05-30 20:02:44 +00:00
ReadBufferFromFile file("/proc/self/maps", buf_size, -1, buf);
while (!file.eof())
{
char * next_pos = find_first_symbols<'\n'>(file.position(), file.buffer().end());
file.position() = next_pos;
if (!file.hasPendingData())
continue;
if (*file.position() == '\n')
{
++num_maps;
++file.position();
}
}
}
2022-06-10 11:22:57 +00:00
if (num_maps > max_map_count * 0.90)
{
msg += fmt::format(
"\nIt looks like that the process is near the limit on number of virtual memory mappings."
"\nCurrent number of mappings (/proc/self/maps): {}."
"\nLimit on number of mappings (/proc/sys/vm/max_map_count): {}."
2020-05-30 20:02:44 +00:00
"\nYou should increase the limit for vm.max_map_count in /etc/sysctl.conf"
"\n",
num_maps, max_map_count);
}
}
catch (...)
{
msg += "\nCannot obtain additional info about memory usage.";
}
2020-05-30 19:40:23 +00:00
#else
(void)msg;
#endif
}
2022-05-03 13:13:47 +00:00
std::string getExtraExceptionInfo(const std::exception & e)
2019-08-05 19:41:20 +00:00
{
String msg;
try
{
2021-05-13 09:32:52 +00:00
if (const auto * file_exception = dynamic_cast<const fs::filesystem_error *>(&e))
2019-08-05 19:41:20 +00:00
{
2021-05-13 09:32:52 +00:00
if (file_exception->code() == std::errc::no_space_on_device)
getNoSpaceLeftInfoMessage(file_exception->path1(), msg);
else
msg += "\nCannot print extra info for Poco::Exception";
2019-08-05 19:41:20 +00:00
}
2020-04-22 05:39:31 +00:00
else if (const auto * errno_exception = dynamic_cast<const DB::ErrnoException *>(&e))
2019-08-05 19:41:20 +00:00
{
2019-08-06 18:54:06 +00:00
if (errno_exception->getErrno() == ENOSPC && errno_exception->getPath())
getNoSpaceLeftInfoMessage(errno_exception->getPath().value(), msg);
else if (errno_exception->code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY
|| errno_exception->code() == ErrorCodes::CANNOT_MREMAP)
getNotEnoughMemoryMessage(msg);
}
else if (dynamic_cast<const std::bad_alloc *>(&e))
{
getNotEnoughMemoryMessage(msg);
2019-08-05 19:41:20 +00:00
}
}
2019-08-06 20:39:07 +00:00
catch (...)
{
2019-08-06 14:46:17 +00:00
msg += "\nCannot print extra info: " + getCurrentExceptionMessage(false, false, false);
2019-08-05 19:41:20 +00:00
}
return msg;
}
2019-08-06 12:51:10 +00:00
std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
2023-01-16 22:20:33 +00:00
{
2023-01-25 20:16:42 +00:00
return getCurrentExceptionMessageAndPattern(with_stacktrace, check_embedded_stacktrace, with_extra_info).text;
2023-01-16 22:20:33 +00:00
}
PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, bool check_embedded_stacktrace /*= false*/, bool with_extra_info /*= true*/)
{
2020-11-10 18:22:26 +00:00
WriteBufferFromOwnString stream;
2023-01-16 22:20:33 +00:00
std::string_view message_format_string;
2013-11-18 19:18:03 +00:00
try
{
throw;
}
catch (const Exception & e)
{
2019-08-05 19:41:20 +00:00
stream << getExceptionMessage(e, with_stacktrace, check_embedded_stacktrace)
2019-08-06 12:51:10 +00:00
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
2019-08-05 19:41:20 +00:00
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
2023-01-16 22:20:33 +00:00
message_format_string = e.tryGetMessageFormatString();
2013-11-18 19:18:03 +00:00
}
catch (const Poco::Exception & e)
{
try
{
2019-04-04 12:34:49 +00:00
stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
2021-07-10 08:18:06 +00:00
<< ", " << e.displayText()
<< (with_stacktrace ? ", Stack trace (when copying this message, always include the lines below):\n\n" + getExceptionStackTraceString(e) : "")
2019-08-06 12:51:10 +00:00
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
2020-01-24 02:38:03 +00:00
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
2013-11-18 19:18:03 +00:00
}
catch (...) {}
}
catch (const std::exception & e)
{
try
{
int status = 0;
auto name = demangle(typeid(e).name(), status);
2014-07-21 09:11:20 +00:00
if (status)
name += " (demangling status: " + toString(status) + ")";
2019-08-05 19:41:20 +00:00
stream << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what()
<< (with_stacktrace ? ", Stack trace (when copying this message, always include the lines below):\n\n" + getExceptionStackTraceString(e) : "")
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
2013-11-18 19:18:03 +00:00
}
catch (...) {}
}
catch (...)
{
try
{
int status = 0;
auto name = demangle(abi::__cxa_current_exception_type()->name(), status);
if (status)
name += " (demangling status: " + toString(status) + ")";
2019-04-04 12:34:49 +00:00
stream << "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
2013-11-18 19:18:03 +00:00
}
catch (...) {}
}
2023-01-16 22:20:33 +00:00
return PreformattedMessage{stream.str(), message_format_string};
2013-11-18 19:18:03 +00:00
}
int getCurrentExceptionCode()
{
try
{
throw;
}
2017-02-07 16:36:12 +00:00
catch (const Exception & e)
{
return e.code();
}
catch (const Poco::Exception &)
2017-02-07 16:36:12 +00:00
{
return ErrorCodes::POCO_EXCEPTION;
}
catch (const std::exception &)
2017-02-07 16:36:12 +00:00
{
return ErrorCodes::STD_EXCEPTION;
}
catch (...)
{
2017-02-07 16:36:12 +00:00
return ErrorCodes::UNKNOWN_EXCEPTION;
}
}
2021-06-18 16:25:19 +00:00
int getExceptionErrorCode(std::exception_ptr e)
{
try
{
std::rethrow_exception(e);
}
2021-06-29 09:25:34 +00:00
catch (const Exception & exception)
2021-06-18 16:25:19 +00:00
{
2021-06-29 09:25:34 +00:00
return exception.code();
2021-06-18 16:25:19 +00:00
}
catch (const Poco::Exception &)
{
return ErrorCodes::POCO_EXCEPTION;
}
catch (const std::exception &)
{
return ErrorCodes::STD_EXCEPTION;
}
catch (...)
{
return ErrorCodes::UNKNOWN_EXCEPTION;
}
}
2015-10-05 05:40:27 +00:00
void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message)
{
try
{
2020-03-18 03:27:32 +00:00
std::rethrow_exception(std::move(e)); // NOLINT
2015-10-05 05:40:27 +00:00
}
catch (...)
{
tryLogCurrentException(log_name, start_of_message);
}
}
void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message)
{
try
{
2020-03-18 03:27:32 +00:00
std::rethrow_exception(std::move(e)); // NOLINT
2015-10-05 05:40:27 +00:00
}
catch (...)
{
tryLogCurrentException(logger, start_of_message);
}
}
std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
2023-01-16 23:11:59 +00:00
{
2023-01-25 20:16:42 +00:00
return getExceptionMessageAndPattern(e, with_stacktrace, check_embedded_stacktrace).text;
2023-01-16 23:11:59 +00:00
}
PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace)
{
2020-11-10 18:22:26 +00:00
WriteBufferFromOwnString stream;
try
{
std::string text = e.displayText();
bool has_embedded_stack_trace = false;
if (check_embedded_stacktrace)
{
auto embedded_stack_trace_pos = text.find("Stack trace");
has_embedded_stack_trace = embedded_stack_trace_pos != std::string::npos;
if (!with_stacktrace && has_embedded_stack_trace)
{
text.resize(embedded_stack_trace_pos);
Poco::trimRightInPlace(text);
}
}
2021-07-10 08:18:06 +00:00
stream << "Code: " << e.code() << ". " << text;
2021-07-10 02:43:53 +00:00
if (!text.empty() && text.back() != '.')
stream << '.';
stream << " (" << ErrorCodes::getName(e.code()) << ")";
if (with_stacktrace && !has_embedded_stack_trace)
stream << ", Stack trace (when copying this message, always include the lines below):\n\n" << e.getStackTraceString();
}
catch (...) {}
2023-01-16 23:11:59 +00:00
return PreformattedMessage{stream.str(), e.tryGetMessageFormatString()};
}
2015-10-05 05:40:27 +00:00
std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace)
{
try
{
2020-03-18 03:27:32 +00:00
std::rethrow_exception(std::move(e)); // NOLINT
2015-10-05 05:40:27 +00:00
}
catch (...)
{
return getCurrentExceptionMessage(with_stacktrace);
}
}
std::string ExecutionStatus::serializeText() const
{
2017-07-31 21:39:24 +00:00
WriteBufferFromOwnString wb;
wb << code << "\n" << escape << message;
return wb.str();
}
void ExecutionStatus::deserializeText(const std::string & data)
{
ReadBufferFromString rb(data);
2017-05-31 14:01:08 +00:00
rb >> code >> "\n" >> escape >> message;
}
2017-07-27 13:11:16 +00:00
bool ExecutionStatus::tryDeserializeText(const std::string & data)
{
try
{
deserializeText(data);
}
catch (...)
{
return false;
}
return true;
}
ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_of_message, bool with_stacktrace)
2023-01-13 10:42:52 +00:00
{
String msg = (start_of_message.empty() ? "" : (start_of_message + ": ")) + getCurrentExceptionMessage(with_stacktrace, true);
2023-01-13 10:42:52 +00:00
return ExecutionStatus(getCurrentExceptionCode(), msg);
}
2021-09-15 18:06:20 +00:00
ExecutionStatus ExecutionStatus::fromText(const std::string & data)
{
ExecutionStatus status;
status.deserializeText(data);
return status;
}
ParsingException::ParsingException() = default;
2020-12-18 01:47:24 +00:00
ParsingException::ParsingException(const std::string & msg, int code)
2020-12-22 22:33:07 +00:00
: Exception(msg, code)
2020-12-18 01:47:24 +00:00
{
}
/// We use additional field formatted_message_ to make this method const.
std::string ParsingException::displayText() const
{
try
{
2022-04-15 23:56:45 +00:00
formatted_message = message();
2022-04-20 14:13:04 +00:00
bool need_newline = false;
2022-04-15 23:56:45 +00:00
if (!file_name.empty())
2022-04-20 14:13:04 +00:00
{
formatted_message += fmt::format(": (in file/uri {})", file_name);
2022-04-20 14:13:04 +00:00
need_newline = true;
}
2022-04-15 23:56:45 +00:00
if (line_number != -1)
2022-04-20 14:13:04 +00:00
{
2022-04-15 23:56:45 +00:00
formatted_message += fmt::format(": (at row {})", line_number);
2022-04-20 14:13:04 +00:00
need_newline = true;
}
if (need_newline)
formatted_message += "\n";
2020-12-18 01:47:24 +00:00
}
2022-04-15 23:56:45 +00:00
catch (...) {}
2020-12-18 01:47:24 +00:00
if (!formatted_message.empty())
2020-12-18 01:47:24 +00:00
{
std::string result = name();
result.append(": ");
result.append(formatted_message);
2020-12-18 01:47:24 +00:00
return result;
}
else
{
return Exception::displayText();
}
}
2010-03-01 16:59:51 +00:00
}