mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
Separate accounting of remote exceptions in system.errors
This commit is contained in:
parent
b39f8cc6ac
commit
259e5ba88e
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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 */);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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))
|
||||
|
Loading…
Reference in New Issue
Block a user