mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #21529 from azat/system.errors-improvements
Add last_error_time/last_error_message/last_error_stacktrace/remote columns for system.errors
This commit is contained in:
commit
4f1f344d4d
@ -7,11 +7,15 @@ Columns:
|
||||
- `name` ([String](../../sql-reference/data-types/string.md)) — name of the error (`errorCodeToName`).
|
||||
- `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — code number of the error.
|
||||
- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — the number of times this error has been happened.
|
||||
- `last_error_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — time when the last error happened.
|
||||
- `last_error_message` ([String](../../sql-reference/data-types/string.md)) — message for the last error.
|
||||
- `last_error_stacktrace` ([String](../../sql-reference/data-types/string.md)) — stacktrace for the last error.
|
||||
- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — remote exception (i.e. received during one of the distributed query).
|
||||
|
||||
**Example**
|
||||
|
||||
``` sql
|
||||
SELECT *
|
||||
SELECT name, code, value
|
||||
FROM system.errors
|
||||
WHERE value > 0
|
||||
ORDER BY code ASC
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <chrono>
|
||||
|
||||
/** Previously, these constants were located in one enum.
|
||||
* But in this case there is a problem: when you add a new constant, you need to recompile
|
||||
@ -562,8 +563,8 @@ namespace ErrorCodes
|
||||
APPLY_FOR_ERROR_CODES(M)
|
||||
#undef M
|
||||
|
||||
constexpr Value END = 3000;
|
||||
std::atomic<Value> values[END + 1]{};
|
||||
constexpr ErrorCode END = 3000;
|
||||
ErrorPairHolder values[END + 1]{};
|
||||
|
||||
struct ErrorCodesNames
|
||||
{
|
||||
@ -578,12 +579,43 @@ namespace ErrorCodes
|
||||
|
||||
std::string_view getName(ErrorCode error_code)
|
||||
{
|
||||
if (error_code >= END)
|
||||
if (error_code < 0 || error_code >= END)
|
||||
return std::string_view();
|
||||
return error_codes_names.names[error_code];
|
||||
}
|
||||
|
||||
ErrorCode end() { return END + 1; }
|
||||
|
||||
void increment(ErrorCode error_code, bool remote, const std::string & message, const std::string & stacktrace)
|
||||
{
|
||||
if (error_code >= end())
|
||||
{
|
||||
/// For everything outside the range, use END.
|
||||
/// (end() is the pointer pass the end, while END is the last value that has an element in values array).
|
||||
error_code = end() - 1;
|
||||
}
|
||||
|
||||
values[error_code].increment(remote, message, stacktrace);
|
||||
}
|
||||
|
||||
void ErrorPairHolder::increment(bool remote, const std::string & message, const std::string & stacktrace)
|
||||
{
|
||||
const auto now = std::chrono::system_clock::now();
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto & error = remote ? value.remote : value.local;
|
||||
|
||||
++error.count;
|
||||
error.message = message;
|
||||
error.stacktrace = stacktrace;
|
||||
error.error_time_ms = std::chrono::duration_cast<std::chrono::milliseconds>(now.time_since_epoch()).count();
|
||||
}
|
||||
ErrorPair ErrorPairHolder::get()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return value;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <stddef.h>
|
||||
#include <cstdint>
|
||||
#include <utility>
|
||||
#include <atomic>
|
||||
#include <mutex>
|
||||
#include <common/types.h>
|
||||
#include <string_view>
|
||||
|
||||
@ -17,30 +17,50 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
/// ErrorCode identifier (index in array).
|
||||
using ErrorCode = size_t;
|
||||
using Value = int;
|
||||
using ErrorCode = int;
|
||||
using Value = size_t;
|
||||
|
||||
/// Get name of error_code by identifier.
|
||||
/// Returns statically allocated string.
|
||||
std::string_view getName(ErrorCode error_code);
|
||||
|
||||
struct Error
|
||||
{
|
||||
/// Number of times Exception with this ErrorCode had been throw.
|
||||
Value count;
|
||||
/// Time of the last error.
|
||||
UInt64 error_time_ms = 0;
|
||||
/// Message for the last error.
|
||||
std::string message;
|
||||
/// Stacktrace for the last error.
|
||||
std::string stacktrace;
|
||||
};
|
||||
struct ErrorPair
|
||||
{
|
||||
Error local;
|
||||
Error remote;
|
||||
};
|
||||
|
||||
/// Thread-safe
|
||||
struct ErrorPairHolder
|
||||
{
|
||||
public:
|
||||
ErrorPair get();
|
||||
void increment(bool remote, const std::string & message, const std::string & stacktrace);
|
||||
|
||||
private:
|
||||
ErrorPair value;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
/// ErrorCode identifier -> current value of error_code.
|
||||
extern std::atomic<Value> values[];
|
||||
extern ErrorPairHolder values[];
|
||||
|
||||
/// Get index just after last error_code identifier.
|
||||
ErrorCode end();
|
||||
|
||||
/// Add value for specified error_code.
|
||||
inline void increment(ErrorCode error_code)
|
||||
{
|
||||
if (error_code >= end())
|
||||
{
|
||||
/// For everything outside the range, use END.
|
||||
/// (end() is the pointer pass the end, while END is the last value that has an element in values array).
|
||||
error_code = end() - 1;
|
||||
}
|
||||
values[error_code].fetch_add(1, std::memory_order_relaxed);
|
||||
}
|
||||
void increment(ErrorCode error_code, bool remote, const std::string & message, const std::string & stacktrace);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -34,9 +34,9 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_MREMAP;
|
||||
}
|
||||
|
||||
/// 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)
|
||||
/// - Aborts the process if error code is LOGICAL_ERROR.
|
||||
/// - Increments error codes statistics.
|
||||
void handle_error_code([[maybe_unused]] const std::string & msg, const std::string & stacktrace, int code, bool remote)
|
||||
{
|
||||
// In debug builds and builds with sanitizers, treat LOGICAL_ERROR as an assertion failure.
|
||||
// Log the message before we fail.
|
||||
@ -47,20 +47,20 @@ void handle_error_code([[maybe_unused]] const std::string & msg, int code)
|
||||
abort();
|
||||
}
|
||||
#endif
|
||||
ErrorCodes::increment(code);
|
||||
ErrorCodes::increment(code, remote, msg, stacktrace);
|
||||
}
|
||||
|
||||
Exception::Exception(const std::string & msg, int code, bool remote_)
|
||||
: Poco::Exception(msg, code)
|
||||
, remote(remote_)
|
||||
{
|
||||
handle_error_code(msg, code);
|
||||
handle_error_code(msg, getStackTraceString(), code, remote);
|
||||
}
|
||||
|
||||
Exception::Exception(const std::string & msg, const Exception & nested, int code)
|
||||
: Poco::Exception(msg, nested, code)
|
||||
{
|
||||
handle_error_code(msg, code);
|
||||
handle_error_code(msg, getStackTraceString(), code, remote);
|
||||
}
|
||||
|
||||
Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Storages/System/StorageSystemErrors.h>
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Interpreters/Context.h>
|
||||
@ -10,30 +11,44 @@ namespace DB
|
||||
NamesAndTypesList StorageSystemErrors::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
{ "name", std::make_shared<DataTypeString>() },
|
||||
{ "code", std::make_shared<DataTypeInt32>() },
|
||||
{ "value", std::make_shared<DataTypeUInt64>() },
|
||||
{ "name", std::make_shared<DataTypeString>() },
|
||||
{ "code", std::make_shared<DataTypeInt32>() },
|
||||
{ "value", std::make_shared<DataTypeUInt64>() },
|
||||
{ "last_error_time", std::make_shared<DataTypeDateTime>() },
|
||||
{ "last_error_message", std::make_shared<DataTypeString>() },
|
||||
{ "last_error_stacktrace", std::make_shared<DataTypeString>() },
|
||||
{ "remote", std::make_shared<DataTypeUInt8>() },
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
void StorageSystemErrors::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
|
||||
{
|
||||
auto add_row = [&](std::string_view name, size_t code, const auto & error, bool remote)
|
||||
{
|
||||
if (error.count || context.getSettingsRef().system_events_show_zero_values)
|
||||
{
|
||||
size_t col_num = 0;
|
||||
res_columns[col_num++]->insert(name);
|
||||
res_columns[col_num++]->insert(code);
|
||||
res_columns[col_num++]->insert(error.count);
|
||||
res_columns[col_num++]->insert(error.error_time_ms / 1000);
|
||||
res_columns[col_num++]->insert(error.message);
|
||||
res_columns[col_num++]->insert(error.stacktrace);
|
||||
res_columns[col_num++]->insert(remote);
|
||||
}
|
||||
};
|
||||
|
||||
for (size_t i = 0, end = ErrorCodes::end(); i < end; ++i)
|
||||
{
|
||||
UInt64 value = ErrorCodes::values[i];
|
||||
const auto & error = ErrorCodes::values[i].get();
|
||||
std::string_view name = ErrorCodes::getName(i);
|
||||
|
||||
if (name.empty())
|
||||
continue;
|
||||
|
||||
if (value || context.getSettingsRef().system_events_show_zero_values)
|
||||
{
|
||||
size_t col_num = 0;
|
||||
res_columns[col_num++]->insert(name);
|
||||
res_columns[col_num++]->insert(i);
|
||||
res_columns[col_num++]->insert(value);
|
||||
}
|
||||
add_row(name, i, error.local, /* remote= */ false);
|
||||
add_row(name, i, error.remote, /* remote= */ true);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
|
||||
|
||||
|
||||
OK
|
||||
UNSUPPORTED_METHOD
|
||||
|
@ -1,4 +1,5 @@
|
||||
SELECT errorCodeToName(toUInt32(-1));
|
||||
SELECT errorCodeToName(-1);
|
||||
SELECT errorCodeToName(600); /* gap in error codes */
|
||||
SELECT errorCodeToName(0);
|
||||
SELECT errorCodeToName(1);
|
||||
|
@ -1 +1,2 @@
|
||||
1
|
||||
local=1
|
||||
remote=1
|
||||
|
@ -4,7 +4,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
prev="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO'")"
|
||||
# local
|
||||
prev="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO' AND NOT remote")"
|
||||
$CLICKHOUSE_CLIENT -q 'SELECT throwIf(1)' >& /dev/null
|
||||
cur="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO'")"
|
||||
echo $((cur - prev))
|
||||
cur="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO' AND NOT remote")"
|
||||
echo local=$((cur - prev))
|
||||
|
||||
# remote
|
||||
prev="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO' AND remote")"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.2', system.one) where throwIf(not dummy)" >& /dev/null
|
||||
cur="$(${CLICKHOUSE_CLIENT} -q "SELECT value FROM system.errors WHERE name = 'FUNCTION_THROW_IF_VALUE_IS_NON_ZERO' AND remote")"
|
||||
echo remote=$((cur - prev))
|
||||
|
@ -739,6 +739,7 @@
|
||||
"01541_max_memory_usage_for_user_long",
|
||||
"01542_dictionary_load_exception_race",
|
||||
"01560_optimize_on_insert_zookeeper",
|
||||
"01545_system_errors", // looks at the difference of values in system.errors
|
||||
"01575_disable_detach_table_of_dictionary",
|
||||
"01593_concurrent_alter_mutations_kill",
|
||||
"01593_concurrent_alter_mutations_kill_many_replicas",
|
||||
|
Loading…
Reference in New Issue
Block a user