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:
Nikolai Kochetov 2021-03-17 11:51:29 +03:00 committed by GitHub
commit 4f1f344d4d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 121 additions and 39 deletions

View File

@ -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

View File

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

View File

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

View File

@ -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)

View File

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

View File

@ -1,4 +1,5 @@
OK
UNSUPPORTED_METHOD

View File

@ -1,4 +1,5 @@
SELECT errorCodeToName(toUInt32(-1));
SELECT errorCodeToName(-1);
SELECT errorCodeToName(600); /* gap in error codes */
SELECT errorCodeToName(0);
SELECT errorCodeToName(1);

View File

@ -1 +1,2 @@
1
local=1
remote=1

View File

@ -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))

View File

@ -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",