Skip to content

Commit 97e0c37

Browse files
committed
WIP
``` ERROR 2024-12-16 12:15:55,115 [shard 0:main] crash_tracker - service.cc:107 - Crash loop detected. Too many consecutive crashes 6, exceeded crash_loop_limit configured value 5. To recover Redpanda from this state, manually remove file at path "/var/lib/redpanda/data/startup_log". Crash loop automatically resets 1h after last crash or with node configuration changes. The following crashes have been recorded: Crash #1 at 2024-12-16 12:14:03 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4 Crash #2 at 2024-12-16 12:14:09 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4 Crash #3 at 2024-12-16 12:14:14 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4 Crash #4 at 2024-12-16 12:14:20 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4 Crash redpanda-data#5 at 2024-12-16 12:14:26 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4 Crash redpanda-data#6 at 2024-12-16 12:14:32 - Segmentation fault on shard 0. Backtrace: 0xab575a3 0x5400cdb 0x5409de2 0x376b120 /lib/x86_64-linux-gnu/libc.so.6+0x4251f /lib/x86_64-linux-gnu/libc.so.6+0x11e88c 0xa95055d 0xa93d12e 0xa8ab47d 0xa8a8316 0xa78cf74 0xa78ae70 0x35a0393 0xadce1b9 /lib/x86_64-linux-gnu/libc.so.6+0x29d8f /lib/x86_64-linux-gnu/libc.so.6+0x29e3f 0x3598be4 ```
1 parent 4343e5e commit 97e0c37

File tree

18 files changed

+851
-129
lines changed

18 files changed

+851
-129
lines changed

src/v/CMakeLists.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -136,6 +136,7 @@ add_subdirectory(datalake)
136136
add_subdirectory(cloud_io)
137137
add_subdirectory(cloud_topics)
138138
add_subdirectory(schema)
139+
add_subdirectory(crash_tracker)
139140

140141
option(ENABLE_GIT_VERSION "Build with Git metadata" OFF)
141142

src/v/config/node_config.h

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -121,6 +121,10 @@ struct node_config final : public config_store {
121121
return data_directory().path / "startup_log";
122122
}
123123

124+
std::filesystem::path crash_report_dir_path() const {
125+
return data_directory().path / "crash_reports";
126+
}
127+
124128
/**
125129
* Return the configured cache path if set, otherwise a default
126130
* path within the data directory.

src/v/crash_tracker/BUILD

Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,41 @@
1+
load("//bazel:build.bzl", "redpanda_cc_library")
2+
3+
redpanda_cc_library(
4+
name = "crash_tracker",
5+
srcs = [
6+
"limiter.cc",
7+
"logger.cc",
8+
"recorder.cc",
9+
"service.cc",
10+
],
11+
hdrs = [
12+
"limiter.h",
13+
"logger.h",
14+
"recorder.h",
15+
"service.h",
16+
"types.h",
17+
],
18+
implementation_deps = [
19+
"//src/v/bytes",
20+
"//src/v/bytes:iobuf",
21+
"//src/v/config",
22+
"//src/v/hashing:xx",
23+
"//src/v/model",
24+
"//src/v/random:generators",
25+
"//src/v/serde",
26+
"//src/v/serde:bytes",
27+
"//src/v/serde:iobuf",
28+
"//src/v/serde:sstring",
29+
"//src/v/serde:vector",
30+
"//src/v/utils:file_io",
31+
"@fmt",
32+
],
33+
include_prefix = "crash_tracker",
34+
visibility = ["//visibility:public"],
35+
deps = [
36+
"//src/v/base",
37+
"//src/v/model",
38+
"//src/v/serde",
39+
"@seastar",
40+
],
41+
)

src/v/crash_tracker/CMakeLists.txt

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
v_cc_library(
2+
NAME crash_tracker
3+
SRCS
4+
limiter.cc
5+
logger.cc
6+
recorder.cc
7+
service.cc
8+
DEPS
9+
Seastar::seastar
10+
v::base
11+
v::bytes
12+
v::model
13+
v::serde
14+
v::hashing
15+
v::random
16+
)

src/v/crash_tracker/limiter.cc

Lines changed: 140 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,140 @@
1+
/*
2+
* Copyright 2024 Redpanda Data, Inc.
3+
*
4+
* Use of this software is governed by the Business Source License
5+
* included in the file licenses/BSL.md
6+
*
7+
* As of the Change Date specified in that file, in accordance with
8+
* the Business Source License, use of this software will be governed
9+
* by the Apache License, Version 2.0
10+
*/
11+
12+
#include "crash_tracker/limiter.h"
13+
14+
#include "base/vassert.h"
15+
#include "config/node_config.h"
16+
#include "crash_tracker/limiter.h"
17+
#include "crash_tracker/logger.h"
18+
#include "crash_tracker/recorder.h"
19+
#include "crash_tracker/service.h"
20+
#include "crash_tracker/types.h"
21+
#include "hashing/xx.h"
22+
#include "model/timestamp.h"
23+
#include "serde/rw/envelope.h"
24+
#include "utils/file_io.h"
25+
26+
#include <seastar/core/seastar.hh>
27+
#include <seastar/util/print_safe.hh>
28+
29+
#include <fmt/chrono.h>
30+
31+
#include <chrono>
32+
#include <system_error>
33+
#include <unistd.h>
34+
35+
using namespace std::chrono_literals;
36+
37+
namespace crash_tracker {
38+
39+
// Crash tracking resets every 1h.
40+
static constexpr model::timestamp_clock::duration crash_reset_duration{1h};
41+
42+
ss::future<> limiter::check_for_crash_loop() const {
43+
auto file_path = config::node().crash_loop_tracker_path();
44+
std::optional<crash_tracker_metadata> maybe_crash_md;
45+
if (
46+
// Tracking is reset every time the broker boots in recovery mode.
47+
!config::node().recovery_mode_enabled()
48+
&& co_await ss::file_exists(file_path.string())) {
49+
// Ok to read the entire file, it contains a serialized uint32_t.
50+
auto buf = co_await read_fully(file_path);
51+
try {
52+
maybe_crash_md = serde::from_iobuf<crash_tracker_metadata>(
53+
std::move(buf));
54+
} catch (const serde::serde_exception&) {
55+
// A malformed log file, ignore and reset it later.
56+
// We truncate it below.
57+
vlog(ctlog.warn, "Ignorning malformed tracker file {}", file_path);
58+
}
59+
}
60+
61+
// Compute the checksum of the current node configuration.
62+
auto current_config = co_await read_fully_to_string(
63+
config::node().get_cfg_file_path());
64+
auto checksum = xxhash_64(current_config.c_str(), current_config.length());
65+
66+
if (maybe_crash_md) {
67+
auto& crash_md = maybe_crash_md.value();
68+
auto& limit = config::node().crash_loop_limit.value();
69+
70+
// Check if it has been atleast 1h since last unsuccessful restart.
71+
// Tracking resets every 1h.
72+
auto time_since_last_start
73+
= model::duration_since_epoch(model::timestamp::now())
74+
- model::duration_since_epoch(crash_md._last_start_ts);
75+
76+
auto crash_limit_ok = !limit || crash_md._crash_count <= limit.value();
77+
auto node_config_changed = crash_md._config_checksum != checksum;
78+
auto tracking_reset = time_since_last_start > crash_reset_duration;
79+
80+
auto ok_to_proceed = crash_limit_ok || node_config_changed
81+
|| tracking_reset;
82+
83+
if (!ok_to_proceed) {
84+
auto crashes = co_await _recorder.get_recorded_crashes();
85+
vlog(
86+
ctlog.error,
87+
"Crash loop detected. Too many consecutive crashes {}, exceeded "
88+
"{} configured value {}. To recover Redpanda from this state, "
89+
"manually remove file at path {}. Crash loop automatically "
90+
"resets 1h after last crash or with node configuration changes. "
91+
"{}",
92+
crash_md._crash_count,
93+
config::node().crash_loop_limit.name(),
94+
limit.value(),
95+
file_path,
96+
crashes.describe());
97+
throw std::runtime_error("Crash loop detected, aborting startup.");
98+
}
99+
100+
vlog(
101+
ctlog.debug,
102+
"Consecutive crashes detected: {} node config changed: {} "
103+
"time based tracking reset: {}",
104+
crash_md._crash_count,
105+
node_config_changed,
106+
tracking_reset);
107+
108+
if (node_config_changed || tracking_reset) {
109+
crash_md._crash_count = 0;
110+
}
111+
}
112+
113+
// Truncate and bump the crash count. We consider a run to be unclean by
114+
// default unless the scheduled cleanup (that runs very late in shutdown)
115+
// resets the file. See schedule_crash_tracker_file_cleanup().
116+
auto new_crash_count = maybe_crash_md
117+
? maybe_crash_md.value()._crash_count + 1
118+
: 1;
119+
crash_tracker_metadata updated{
120+
._crash_count = new_crash_count,
121+
._config_checksum = checksum,
122+
._last_start_ts = model::timestamp::now()};
123+
co_await write_fully(file_path, serde::to_iobuf(updated));
124+
co_await ss::sync_directory(
125+
config::node().data_directory.value().as_sstring());
126+
}
127+
128+
ss::future<> limiter::record_clean_shutdown() const {
129+
auto file = config::node().crash_loop_tracker_path().string();
130+
if (co_await ss::file_exists(file)) {
131+
co_await ss::remove_file(file);
132+
co_await ss::sync_directory(
133+
config::node().data_directory().as_sstring());
134+
vlog(ctlog.debug, "Deleted crash loop tracker file: {}", file);
135+
}
136+
137+
co_return;
138+
}
139+
140+
} // namespace crash_tracker

src/v/crash_tracker/limiter.h

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,35 @@
1+
/*
2+
* Copyright 2024 Redpanda Data, Inc.
3+
*
4+
* Use of this software is governed by the Business Source License
5+
* included in the file licenses/BSL.md
6+
*
7+
* As of the Change Date specified in that file, in accordance with
8+
* the Business Source License, use of this software will be governed
9+
* by the Apache License, Version 2.0
10+
*/
11+
12+
#pragma once
13+
14+
#include "base/seastarx.h"
15+
#include "bytes/iobuf.h"
16+
#include "crash_tracker/recorder.h"
17+
#include "crash_tracker/types.h"
18+
#include "model/timestamp.h"
19+
#include "serde/envelope.h"
20+
21+
namespace crash_tracker {
22+
23+
// Limits the number of restarts to a configured amount
24+
class limiter {
25+
public:
26+
explicit limiter(const recorder& recorder)
27+
: _recorder(recorder) {};
28+
ss::future<> check_for_crash_loop() const;
29+
ss::future<> record_clean_shutdown() const;
30+
31+
private:
32+
const recorder& _recorder;
33+
};
34+
35+
} // namespace crash_tracker

src/v/crash_tracker/logger.cc

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,17 @@
1+
/*
2+
* Copyright 2024 Redpanda Data, Inc.
3+
*
4+
* Use of this software is governed by the Business Source License
5+
* included in the file licenses/BSL.md
6+
*
7+
* As of the Change Date specified in that file, in accordance with
8+
* the Business Source License, use of this software will be governed
9+
* by the Apache License, Version 2.0
10+
*/
11+
#include "logger.h"
12+
13+
namespace crash_tracker {
14+
15+
seastar::logger ctlog("crash_tracker");
16+
17+
} // namespace crash_tracker

src/v/crash_tracker/logger.h

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,20 @@
1+
/*
2+
* Copyright 2024 Redpanda Data, Inc.
3+
*
4+
* Use of this software is governed by the Business Source License
5+
* included in the file licenses/BSL.md
6+
*
7+
* As of the Change Date specified in that file, in accordance with
8+
* the Business Source License, use of this software will be governed
9+
* by the Apache License, Version 2.0
10+
*/
11+
12+
#pragma once
13+
14+
#include "base/seastarx.h"
15+
16+
#include <seastar/util/log.hh>
17+
18+
namespace crash_tracker {
19+
extern ss::logger ctlog;
20+
} // namespace crash_tracker

0 commit comments

Comments
 (0)