Skip to content

Commit

Permalink
Merge pull request #24787 from pgellert/crashlog/crash-loop-sleep
Browse files Browse the repository at this point in the history
CORE-8616 redpanda: configurable sleep on crash loop
  • Loading branch information
pgellert authored Jan 15, 2025
2 parents 5b75121 + 3192708 commit 3e19fa1
Show file tree
Hide file tree
Showing 6 changed files with 61 additions and 11 deletions.
10 changes: 10 additions & 0 deletions src/v/config/node_config.cc
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,16 @@ node_config::node_config() noexcept
"broker-properties/#crash_loop_limit.",
{.visibility = visibility::user},
5) // default value
, crash_loop_sleep_sec(
*this,
"crash_loop_sleep_sec",
"The amount of time the broker sleeps before terminating the process "
"when it reaches the number of consecutive times a broker can crash. For "
"more information, see "
"https://docs.redpanda.com/current/reference/properties/"
"broker-properties/#crash_loop_limit.",
{.visibility = visibility::user},
std::nullopt)
, upgrade_override_checks(
*this,
"upgrade_override_checks",
Expand Down
1 change: 1 addition & 0 deletions src/v/config/node_config.h
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ struct node_config final : public config_store {
deprecated_property enable_central_config;

property<std::optional<uint32_t>> crash_loop_limit;
property<std::optional<std::chrono::seconds>> crash_loop_sleep_sec;

// If true, permit any version of redpanda to start, even
// if potentially incompatible with existing system state.
Expand Down
16 changes: 14 additions & 2 deletions src/v/redpanda/application.cc
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,7 @@
#include <seastar/core/seastar.hh>
#include <seastar/core/sharded.hh>
#include <seastar/core/shared_ptr.hh>
#include <seastar/core/sleep.hh>
#include <seastar/core/smp.hh>
#include <seastar/core/thread.hh>
#include <seastar/json/json_elements.hh>
Expand Down Expand Up @@ -501,7 +502,7 @@ int application::run(int ac, char** av) {
hydrate_config(cfg);
initialize();
check_environment();
check_for_crash_loop();
check_for_crash_loop(app_signal.abort_source());
setup_metrics();
wire_up_and_start(app_signal);
post_start_tasks();
Expand Down Expand Up @@ -1030,7 +1031,7 @@ void application::check_environment() {
/// the broker last failed to start. This metadata is tracked in the
/// tracker file. This is to prevent on disk state from piling up in
/// each unclean run and creating more state to recover for the next run.
void application::check_for_crash_loop() {
void application::check_for_crash_loop(ss::abort_source& as) {
if (config::node().developer_mode()) {
// crash loop tracking has value only in long running clusters
// that can potentially accumulate state across restarts.
Expand Down Expand Up @@ -1087,6 +1088,17 @@ void application::check_for_crash_loop() {
config::node().crash_loop_limit.name(),
limit.value(),
file_path);

const auto crash_loop_sleep_val
= config::node().crash_loop_sleep_sec.value();
if (crash_loop_sleep_val) {
vlog(
_log.info,
"Sleeping for {} seconds before terminating...",
*crash_loop_sleep_val / 1s);
ss::sleep_abortable(*crash_loop_sleep_val, as).get();
}

throw std::runtime_error("Crash loop detected, aborting startup.");
}

Expand Down
2 changes: 1 addition & 1 deletion src/v/redpanda/application.h
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ class application {
void wire_up_and_start(::stop_signal&, bool test_mode = false);
void post_start_tasks();

void check_for_crash_loop();
void check_for_crash_loop(ss::abort_source&);
void schedule_crash_tracker_file_cleanup();

explicit application(ss::sstring = "main");
Expand Down
4 changes: 4 additions & 0 deletions tests/rptest/services/redpanda.py
Original file line number Diff line number Diff line change
Expand Up @@ -2657,6 +2657,10 @@ def set_extra_node_conf(self, node, conf):
assert node in self.nodes, f"Node {node.account.hostname} is not started"
self._extra_node_conf[node] = conf

def add_extra_node_conf(self, node, conf):
assert node in self.nodes, f"Node {node.account.hostname} is not started"
self._extra_node_conf[node] = {**self._extra_node_conf[node], **conf}

def set_security_settings(self, settings):
self._security = settings
self._init_tls()
Expand Down
39 changes: 31 additions & 8 deletions tests/rptest/tests/crash_loop_checks_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,9 @@
from rptest.services.cluster import cluster
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.redpanda import RedpandaService
from rptest.util import expect_exception
from rptest.services.redpanda import LoggingConfig
from ducktape.errors import TimeoutError


class CrashLoopChecksTest(RedpandaTest):
Expand All @@ -32,14 +35,15 @@ class CrashLoopChecksTest(RedpandaTest):
CRASH_LOOP_TRACKER_FILE = f"{RedpandaService.DATA_DIR}/startup_log"

def __init__(self, test_context):
super(CrashLoopChecksTest,
self).__init__(test_context=test_context,
num_brokers=1,
extra_node_conf={
"crash_loop_limit":
CrashLoopChecksTest.CRASH_LOOP_LIMIT,
"developer_mode": False
})
super(CrashLoopChecksTest, self).__init__(
test_context=test_context,
num_brokers=1,
extra_node_conf={
"crash_loop_limit": CrashLoopChecksTest.CRASH_LOOP_LIMIT,
"developer_mode": False
},
log_config=LoggingConfig('info', logger_levels={'main': 'debug'}),
)

def remove_crash_loop_tracker_file(self, broker):
broker.account.ssh(
Expand Down Expand Up @@ -108,3 +112,22 @@ def test_crash_loop_tracker_reset_via_recovery_mode(self):
# stop + restart without recovery mode.
self.redpanda.stop_node(broker)
self.redpanda.start_node(broker)

@cluster(num_nodes=1, log_allow_list=CRASH_LOOP_LOG)
def test_crash_loop_sleep(self):
broker = self.redpanda.nodes[0]

self.redpanda.add_extra_node_conf(broker, {"crash_loop_sleep_sec": 3})
self.redpanda.restart_nodes(broker)

for _ in range(CrashLoopChecksTest.CRASH_LOOP_LIMIT):
self.redpanda.signal_redpanda(node=broker)
self.redpanda.start_node(broker)
self.redpanda.signal_redpanda(node=broker)

# Expect the redpanda process to sleep for crash_loop_sleep_sec
self.redpanda.start_node(node=broker, expect_fail=True)
assert self.redpanda.search_log_node(broker,
"Too many consecutive crashes")
assert self.redpanda.search_log_node(
broker, "Sleeping for 3 seconds before terminating...")

0 comments on commit 3e19fa1

Please sign in to comment.