Skip to content

Commit

Permalink
Merge pull request #24854 from pgellert/crashlog/record-startup-exc
Browse files Browse the repository at this point in the history
CORE-8618 crash_tracker: record uncaught startup exceptions
  • Loading branch information
pgellert authored Jan 27, 2025
2 parents 8a1124b + 5d3e146 commit da553ee
Show file tree
Hide file tree
Showing 14 changed files with 490 additions and 5 deletions.
4 changes: 4 additions & 0 deletions src/v/config/node_config.h
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,10 @@ struct node_config final : public config_store {
return data_directory().path / "startup_log";
}

std::filesystem::path crash_report_dir_path() const {
return data_directory().path / "crash_reports";
}

/**
* Return the configured cache path if set, otherwise a default
* path within the data directory.
Expand Down
3 changes: 3 additions & 0 deletions src/v/crash_tracker/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -5,12 +5,15 @@ redpanda_cc_library(
srcs = [
"limiter.cc",
"logger.cc",
"prepared_writer.cc",
"recorder.cc",
"service.cc",
"types.cc",
],
hdrs = [
"limiter.h",
"logger.h",
"prepared_writer.h",
"recorder.h",
"service.h",
"types.h",
Expand Down
2 changes: 2 additions & 0 deletions src/v/crash_tracker/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,10 @@ v_cc_library(
SRCS
limiter.cc
logger.cc
prepared_writer.cc
recorder.cc
service.cc
types.cc
DEPS
Seastar::seastar
v::base
Expand Down
2 changes: 1 addition & 1 deletion src/v/crash_tracker/limiter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ ss::future<> limiter::check_for_crash_loop(ss::abort_source& as) const {
co_await ss::sleep_abortable(*crash_loop_sleep_val, as);
}

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

vlog(
Expand Down
184 changes: 184 additions & 0 deletions src/v/crash_tracker/prepared_writer.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,184 @@
/*
* Copyright 2025 Redpanda Data, Inc.
*
* Use of this software is governed by the Business Source License
* included in the file licenses/BSL.md
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/

#include "crash_tracker/prepared_writer.h"

#include "crash_tracker/logger.h"
#include "crash_tracker/types.h"
#include "hashing/xx.h"
#include "model/timestamp.h"

#include <seastar/core/file-types.hh>
#include <seastar/core/file.hh>
#include <seastar/core/seastar.hh>
#include <seastar/core/sleep.hh>
#include <seastar/util/print_safe.hh>

#include <fmt/chrono.h>

#include <chrono>
#include <fcntl.h>
#include <system_error>
#include <unistd.h>

using namespace std::chrono_literals;

namespace crash_tracker {

std::ostream& operator<<(std::ostream& os, prepared_writer::state s) {
switch (s) {
case prepared_writer::state::uninitialized:
return os << "uninitialized";
case prepared_writer::state::initialized:
return os << "initialized";
case prepared_writer::state::filled:
return os << "filled";
case prepared_writer::state::written:
return os << "written";
case prepared_writer::state::released:
return os << "released";
}
}

ss::future<>
prepared_writer::initialize(std::filesystem::path crash_file_path) {
vassert(_state == state::uninitialized, "Unexpected state: {}", _state);
_crash_report_file_name = std::move(crash_file_path);

_serde_output.reserve_memory(crash_description::serde_size_overestimate);

// Create the crash recorder file
auto f = co_await ss::open_file_dma(
_crash_report_file_name.c_str(),
ss::open_flags::create | ss::open_flags::rw | ss::open_flags::truncate
| ss::open_flags::exclusive);
co_await f.close();

// Open the crash recorder file using ::open().
// We need to use the low level open() function here instead of the seastar
// API or higher-level C++ primitives because we need to be able to
// manipulate the file using async-signal-safe, allocation-free functions
// inside signal handlers.
_fd = ::open(_crash_report_file_name.c_str(), O_WRONLY);
if (_fd == -1) {
throw std::system_error(
errno,
std::system_category(),
fmt::format(
"Failed to open {} to record crash reason",
_crash_report_file_name));
}

// Update _state as the last step in initialize() to make earlier changes
// visible across threads.
_state = state::initialized;
}

crash_description* prepared_writer::fill() {
// Note: this CAS serves two purposes:
// 1. Ensures the visibility of the changes made in initialize()
// 2. Ensures only a single owner can win the race for the prepared_writer
// with competing calls of fill() and release() across threads.
auto before = state::initialized;
auto success = _state.compare_exchange_strong(before, state::filled);
if (!success) {
// The old value of _state could have been anything but uninitialized
// because of competing calls of fill(), write() or release() on other
// threads. These competing calls are unlikely but possible. Example: a
// sigabrt signal handler calls fill() on thread T3 while a segfault
// signal handler calls fill() on thread T2.
vassert(
before != state::uninitialized,
"fill() must be called after initialize(). Unexpected state: {}",
before);
return nullptr;
}
_prepared_cd.crash_time = model::timestamp::now();

return &_prepared_cd;
}

void prepared_writer::write() {
auto before = state::filled;
auto success = _state.compare_exchange_strong(before, state::written);
vassert(
success,
"write() must be called after a fill() that returned a non-null value. "
"Unexpected state: {}",
before);

if (try_write_crash()) {
constexpr static std::string_view success
= "Recorded crash reason to crash file.\n";
ss::print_safe(success.data(), success.size());
} else {
constexpr static std::string_view failure
= "Failed to record crash reason to crash file.\n";
ss::print_safe(failure.data(), failure.size());
}
}

bool prepared_writer::try_write_crash() {
bool success = true;
serde::write(_serde_output, std::move(_prepared_cd));

for (const auto& frag : _serde_output) {
size_t written = 0;
while (written < frag.size()) {
auto res = ::write(
_fd, frag.get() + written, frag.size() - written);
if (res == -1 && errno == EINTR) {
// EINTR is retriable
continue;
} else if (res == -1) {
// Return that writing the crash failed but try to continue to
// write later fragments as much information as possible
success = false;
break;
} else {
written += res;
}
}
}

::fsync(_fd);

return success;
}

ss::future<> prepared_writer::release() {
// Note: this CAS ensures only a single owner can win the race for the
// prepared_writer with competing calls of fill() and release() across
// threads.
auto before = state::initialized;
auto success = _state.compare_exchange_strong(before, state::released);
if (!success) {
// Sanity check that the CAS was unsuccessful because of a race with
// fill() and not because the prepared_writer was not initialized.
vassert(
before != state::uninitialized,
"release() must be called after initialize(). Unexpected state: {}",
before);

// If another call to fill() won the race, release() is a noop
co_return;
}

::close(_fd);
co_await ss::remove_file(_crash_report_file_name.c_str());
vlog(ctlog.debug, "Deleted crash report file: {}", _crash_report_file_name);

_state = state::released;

co_return;
}

} // namespace crash_tracker
75 changes: 75 additions & 0 deletions src/v/crash_tracker/prepared_writer.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Copyright 2025 Redpanda Data, Inc.
*
* Use of this software is governed by the Business Source License
* included in the file licenses/BSL.md
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/

#pragma once

#include "base/seastarx.h"
#include "bytes/iobuf.h"
#include "crash_tracker/types.h"

namespace crash_tracker {

// prepared_writer is a thread-safe helper for writing a crash_description to a
// file in an async-signal safe way. The state transition diagram for the object
// is below. Note that fill() may race with other calls of fill() or with
// release(), and the class safely breaks this race. Calling fill() returns a
// nullptr if it lost the race (i.e. if the object is already in
// filled/written/released state). Calling release() is a noop if it lost the
// race (i.e. if the object is already in filled/written state).
//
// clang-format off
// +---------------+ initialize() +-------------+ fill() +--------+ write() +---------+
// | uninitialized +---------------->| initialized +---------->| filled +----------->| written |
// +---------------+ +------+------+ +--------+ +---------+
// |
// |
// |
// |
// |
// | release() +----------+
// +-------------------------------------->| released |
// +----------+
// clang-format on
class prepared_writer {
public:
ss::future<> initialize(std::filesystem::path);
ss::future<> release();

/// Async-signal safe
/// May return nullptr if the prepared_writer has already been consumed
crash_description* fill();

/// Async-signal safe
/// Must be called after a fill() that returned a non-null value
void write();

private:
enum class state { uninitialized, initialized, filled, written, released };
friend std::ostream& operator<<(std::ostream&, state);

// Returns true on success, false on failure
bool try_write_crash();

std::atomic<state> _state{state::uninitialized};

// We want to avoid taking locks during signal handling. An atomic enum with
// a few states should be lock-free implementable on the platforms redpanda
// supports, but if this check ever fails we could change the type of the
// enum class to an enum or integer.
static_assert(std::atomic<state>::is_always_lock_free);

crash_description _prepared_cd;
iobuf _serde_output;
std::filesystem::path _crash_report_file_name;
int _fd{0};
};

} // namespace crash_tracker
Loading

0 comments on commit da553ee

Please sign in to comment.