Skip to content

Commit

Permalink
Merge pull request ClickHouse#65381 from pamarcos/system-error-log
Browse files Browse the repository at this point in the history
Add system.error_log
  • Loading branch information
pamarcos authored Jun 21, 2024
2 parents 6dd6755 + 71f8937 commit 932e4bf
Show file tree
Hide file tree
Showing 41 changed files with 514 additions and 135 deletions.
4 changes: 2 additions & 2 deletions docker/test/stateless/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -285,7 +285,7 @@ stop_logs_replication

# Try to get logs while server is running
failed_to_save_logs=0
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log
do
err=$(clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.gz' format TSVWithNamesAndTypes")
echo "$err"
Expand Down Expand Up @@ -339,7 +339,7 @@ if [ $failed_to_save_logs -ne 0 ]; then
# directly
# - even though ci auto-compress some files (but not *.tsv) it does this only
# for files >64MB, we want this files to be compressed explicitly
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log
do
clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
Expand Down
34 changes: 33 additions & 1 deletion docs/en/operations/server-configuration-parameters/settings.md
Original file line number Diff line number Diff line change
Expand Up @@ -954,6 +954,38 @@ Or it can be set in hex:

Everything mentioned above can be applied for `aes_256_gcm_siv` (but the key must be 32 bytes long).

## error_log {#error_log}

It is disabled by default.

**Enabling**

To manually turn on error history collection [`system.error_log`](../../operations/system-tables/error_log.md), create `/etc/clickhouse-server/config.d/error_log.xml` with the following content:

``` xml
<clickhouse>
<error_log>
<database>system</database>
<table>error_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
<max_size_rows>1048576</max_size_rows>
<reserved_size_rows>8192</reserved_size_rows>
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
<flush_on_crash>false</flush_on_crash>
</error_log>
</clickhouse>
```

**Disabling**

To disable `error_log` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_error_log.xml` with the following content:

``` xml
<clickhouse>
<error_log remove="1" />
</clickhouse>
```

## custom_settings_prefixes {#custom_settings_prefixes}

Expand Down Expand Up @@ -1917,7 +1949,7 @@ For more information, see the MergeTreeSettings.h header file.

## metric_log {#metric_log}

It is enabled by default. If it`s not, you can do this manually.
It is disabled by default.

**Enabling**

Expand Down
39 changes: 39 additions & 0 deletions docs/en/operations/system-tables/error_log.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
---
slug: /en/operations/system-tables/error_log
---
# error_log

Contains history of error values from table `system.errors`, periodically flushed to disk.

Columns:
- `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time.
- `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — Code number of the error.
- `error` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) - Name of the error.
- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of times this error happened.
- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Remote exception (i.e. received during one of the distributed queries).

**Example**

``` sql
SELECT * FROM system.error_log LIMIT 1 FORMAT Vertical;
```

``` text
Row 1:
──────
hostname: clickhouse.eu-central1.internal
event_date: 2024-06-18
event_time: 2024-06-18 07:32:39
code: 999
error: KEEPER_EXCEPTION
value: 2
remote: 0
```

**See also**

- [error_log setting](../../operations/server-configuration-parameters/settings.md#error_log) — Enabling and disabling the setting.
- [system.errors](../../operations/system-tables/errors.md) — Contains error codes with the number of times they have been triggered.
- [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring.
12 changes: 12 additions & 0 deletions programs/server/config.xml
Original file line number Diff line number Diff line change
Expand Up @@ -1155,6 +1155,18 @@
<flush_on_crash>false</flush_on_crash>
</metric_log>

<!-- Error log contains rows with current values of errors collected with "collect_interval_milliseconds" interval. -->
<error_log>
<database>system</database>
<table>error_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<max_size_rows>1048576</max_size_rows>
<reserved_size_rows>8192</reserved_size_rows>
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
<flush_on_crash>false</flush_on_crash>
</error_log>

<!--
Asynchronous metric log contains values of metrics from
system.asynchronous_metrics.
Expand Down
7 changes: 7 additions & 0 deletions programs/server/config.yaml.example
Original file line number Diff line number Diff line change
Expand Up @@ -726,6 +726,13 @@ metric_log:
flush_interval_milliseconds: 7500
collect_interval_milliseconds: 1000

# Error log contains rows with current values of errors collected with "collect_interval_milliseconds" interval.
error_log:
database: system
table: error_log
flush_interval_milliseconds: 7500
collect_interval_milliseconds: 1000

# Asynchronous metric log contains values of metrics from
# system.asynchronous_metrics.
asynchronous_metric_log:
Expand Down
4 changes: 1 addition & 3 deletions src/Common/ErrorCodes.h
Original file line number Diff line number Diff line change
@@ -1,8 +1,6 @@
#pragma once

#include <cstddef>
#include <cstdint>
#include <utility>
#include <mutex>
#include <string_view>
#include <vector>
Expand Down Expand Up @@ -35,7 +33,7 @@ namespace ErrorCodes

struct Error
{
/// Number of times Exception with this ErrorCode had been throw.
/// Number of times Exception with this ErrorCode has been thrown.
Value count = 0;
/// Time of the last error.
UInt64 error_time_ms = 0;
Expand Down
1 change: 1 addition & 0 deletions src/Common/SystemLogBase.cpp
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/CrashLog.h>
#include <Interpreters/ErrorLog.h>
#include <Interpreters/MetricLog.h>
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <Interpreters/PartLog.h>
Expand Down
5 changes: 2 additions & 3 deletions src/Common/SystemLogBase.h
Original file line number Diff line number Diff line change
@@ -1,9 +1,7 @@
#pragma once

#include <atomic>
#include <condition_variable>
#include <memory>
#include <thread>
#include <vector>
#include <base/types.h>

Expand Down Expand Up @@ -32,7 +30,8 @@
M(FilesystemReadPrefetchesLogElement) \
M(AsynchronousInsertLogElement) \
M(BackupLogElement) \
M(BlobStorageLogElement)
M(BlobStorageLogElement) \
M(ErrorLogElement)

namespace Poco
{
Expand Down
2 changes: 0 additions & 2 deletions src/Interpreters/AsynchronousMetricLog.h
Original file line number Diff line number Diff line change
Expand Up @@ -8,8 +8,6 @@
#include <Core/NamesAndAliases.h>
#include <Storages/ColumnsDescription.h>

#include <vector>
#include <atomic>
#include <ctime>


Expand Down
123 changes: 123 additions & 0 deletions src/Interpreters/ErrorLog.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/ErrorLog.h>
#include <base/getFQDNOrHostName.h>
#include <Common/DateLUTImpl.h>
#include <Common/ThreadPool.h>
#include <Common/ErrorCodes.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/parseQuery.h>

#include <vector>

namespace DB
{

ColumnsDescription ErrorLogElement::getColumnsDescription()
{
ParserCodec codec_parser;
return ColumnsDescription {
{
"hostname",
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Hostname of the server executing the query."
},
{
"event_date",
std::make_shared<DataTypeDate>(),
parseQuery(codec_parser, "(Delta(2), ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Event date."
},
{
"event_time",
std::make_shared<DataTypeDateTime>(),
parseQuery(codec_parser, "(Delta(4), ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Event time."
},
{
"code",
std::make_shared<DataTypeInt32>(),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Error code."
},
{
"error",
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Error name."
},
{
"value",
std::make_shared<DataTypeUInt64>(),
parseQuery(codec_parser, "(ZSTD(3))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Number of errors happened in time interval."
},
{
"remote",
std::make_shared<DataTypeUInt8>(),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Remote exception (i.e. received during one of the distributed queries)."
}
};
}

void ErrorLogElement::appendToBlock(MutableColumns & columns) const
{
size_t column_idx = 0;

columns[column_idx++]->insert(getFQDNOrHostName());
columns[column_idx++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[column_idx++]->insert(event_time);
columns[column_idx++]->insert(code);
columns[column_idx++]->insert(ErrorCodes::getName(code));
columns[column_idx++]->insert(value);
columns[column_idx++]->insert(remote);
}

struct ValuePair
{
UInt64 local = 0;
UInt64 remote = 0;
};

void ErrorLog::stepFunction(TimePoint current_time)
{
/// Static lazy initialization to avoid polluting the header with implementation details
static std::vector<ValuePair> previous_values(ErrorCodes::end());

auto event_time = std::chrono::system_clock::to_time_t(current_time);

for (ErrorCodes::ErrorCode code = 0, end = ErrorCodes::end(); code < end; ++code)
{
const auto & error = ErrorCodes::values[code].get();
if (error.local.count != previous_values.at(code).local)
{
ErrorLogElement local_elem {
.event_time=event_time,
.code=code,
.value=error.local.count - previous_values.at(code).local,
.remote=false
};
this->add(std::move(local_elem));
previous_values[code].local = error.local.count;
}
if (error.remote.count != previous_values.at(code).remote)
{
ErrorLogElement remote_elem {
.event_time=event_time,
.code=code,
.value=error.remote.count - previous_values.at(code).remote,
.remote=true
};
this->add(std::move(remote_elem));
previous_values[code].remote = error.remote.count;
}
}
}

}
39 changes: 39 additions & 0 deletions src/Interpreters/ErrorLog.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
#pragma once

#include <Interpreters/SystemLog.h>
#include <Interpreters/PeriodicLog.h>
#include <Common/ErrorCodes.h>
#include <Core/NamesAndTypes.h>
#include <Core/NamesAndAliases.h>
#include <Storages/ColumnsDescription.h>


namespace DB
{

/** ErrorLog is a log of error values measured at regular time interval.
*/

struct ErrorLogElement
{
time_t event_time{};
ErrorCodes::ErrorCode code{};
ErrorCodes::Value value{};
bool remote{};

static std::string name() { return "ErrorLog"; }
static ColumnsDescription getColumnsDescription();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const;
};


class ErrorLog : public PeriodicLog<ErrorLogElement>
{
using PeriodicLog<ErrorLogElement>::PeriodicLog;

protected:
void stepFunction(TimePoint current_time) override;
};

}
Loading

0 comments on commit 932e4bf

Please sign in to comment.