Separate accounting of remote exceptions in system.errors

This commit is contained in:
Azat Khuzhin 2021-03-08 22:05:51 +03:00
parent b39f8cc6ac
commit 259e5ba88e
7 changed files with 49 additions and 24 deletions

View File

@ -7,11 +7,12 @@ 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.
- `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

@ -563,7 +563,7 @@ namespace ErrorCodes
#undef M
constexpr Value END = 3000;
std::atomic<Value> values[END + 1]{};
ValuePair values[END + 1]{};
struct ErrorCodesNames
{
@ -585,7 +585,7 @@ namespace ErrorCodes
ErrorCode end() { return END + 1; }
void increment(ErrorCode error_code)
void increment(ErrorCode error_code, bool remote)
{
if (error_code >= end())
{
@ -593,7 +593,10 @@ namespace ErrorCodes
/// (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);
if (remote)
values[error_code].remote.fetch_add(1, std::memory_order_relaxed);
else
values[error_code].local.fetch_add(1, std::memory_order_relaxed);
}
}

View File

@ -25,13 +25,18 @@ namespace ErrorCodes
std::string_view getName(ErrorCode error_code);
/// ErrorCode identifier -> current value of error_code.
extern std::atomic<Value> values[];
struct ValuePair
{
std::atomic<Value> local;
std::atomic<Value> remote;
};
extern ValuePair values[];
/// Get index just after last error_code identifier.
ErrorCode end();
/// Add value for specified error_code.
void increment(ErrorCode error_code);
void increment(ErrorCode error_code, bool remote);
}
}

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, 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);
}
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, 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, code, remote);
}
Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc)

View File

@ -13,27 +13,35 @@ NamesAndTypesList StorageSystemErrors::getNamesAndTypes()
{ "name", std::make_shared<DataTypeString>() },
{ "code", std::make_shared<DataTypeInt32>() },
{ "value", std::make_shared<DataTypeUInt64>() },
{ "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, size_t value, bool remote)
{
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(code);
res_columns[col_num++]->insert(value);
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];
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, 0 /* remote=0 */);
add_row(name, i, error.remote, 1 /* remote=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))