Skip to content

Commit

Permalink
Merge pull request redpanda-data#23587 from michael-redpanda/CORE-772…
Browse files Browse the repository at this point in the history
…5-db-metrics

CORE-7725 debug bundle metrics
  • Loading branch information
michael-redpanda authored Oct 3, 2024
2 parents c2c238f + 3d41dfc commit 1442931
Show file tree
Hide file tree
Showing 11 changed files with 366 additions and 30 deletions.
21 changes: 21 additions & 0 deletions src/v/debug_bundle/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ redpanda_cc_library(
],
implementation_deps = [
":metadata",
":probe",
":utils",
"//src/v/bytes:iostream",
"//src/v/crypto",
Expand Down Expand Up @@ -137,3 +138,23 @@ redpanda_cc_library(
"//src/v/bytes",
],
)

redpanda_cc_library(
name = "probe",
srcs = [
"probe.cc",
],
hdrs = [
"probe.h",
],
implementation_deps = [
"//src/v/config",
"@seastar",
],
include_prefix = "debug_bundle",
visibility = ["//visibility:private"],
deps = [
":types",
"//src/v/metrics",
],
)
1 change: 1 addition & 0 deletions src/v/debug_bundle/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ v_cc_library(
SRCS
debug_bundle_service.cc
error.cc
probe.cc
types.cc
utils.cc
DEPS
Expand Down
30 changes: 24 additions & 6 deletions src/v/debug_bundle/debug_bundle_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
#include "crypto/types.h"
#include "debug_bundle/error.h"
#include "debug_bundle/metadata.h"
#include "debug_bundle/probe.h"
#include "debug_bundle/types.h"
#include "debug_bundle/utils.h"
#include "ssx/future-util.h"
Expand Down Expand Up @@ -197,7 +198,8 @@ class service::debug_bundle_process {
, _cerr(
po.has_value() ? std::move(po.value().cerr)
: chunked_vector<ss::sstring>{})
, _created_time(md.get_created_at()) {}
, _created_time(md.get_created_at())
, _finished_time(md.get_finished_at()) {}

debug_bundle_process() = delete;
debug_bundle_process(debug_bundle_process&&) = default;
Expand Down Expand Up @@ -225,6 +227,8 @@ class service::debug_bundle_process {
vassert(
_rpk_process != nullptr,
"RPK process should be created if calling wait()");
auto set_finish_time = ss::defer(
[this] { _finished_time = clock::now(); });
try {
co_return _wait_result.emplace(co_await _rpk_process->wait());
} catch (const std::exception& e) {
Expand Down Expand Up @@ -261,6 +265,7 @@ class service::debug_bundle_process {
const chunked_vector<ss::sstring>& cout() const { return _cout; }
const chunked_vector<ss::sstring>& cerr() const { return _cerr; }
clock::time_point created_time() const { return _created_time; }
clock::time_point finished_time() const { return _finished_time; }
ss::experimental::process::wait_status get_wait_result() const {
vassert(_wait_result.has_value(), "wait_result must have been set");
return _wait_result.value();
Expand All @@ -275,6 +280,7 @@ class service::debug_bundle_process {
chunked_vector<ss::sstring> _cout;
chunked_vector<ss::sstring> _cerr;
clock::time_point _created_time;
clock::time_point _finished_time;
};

service::service(storage::kvstore* kvstore)
Expand Down Expand Up @@ -304,6 +310,9 @@ ss::future<> service::start() {
_rpk_path_binding().native());
}

_probe = std::make_unique<probe>();
_probe->setup_metrics();

co_await maybe_reload_previous_run();

lg.debug("Service started");
Expand All @@ -326,6 +335,7 @@ ss::future<> service::stop() {
}
co_await _gate.close();
_rpk_process.reset(nullptr);
_probe.reset(nullptr);
}

ss::future<result<void>> service::initiate_rpk_debug_bundle_collection(
Expand Down Expand Up @@ -416,13 +426,11 @@ ss::future<result<void>> service::initiate_rpk_debug_bundle_collection(
ssx::spawn_with_gate(_gate, [this, job_id]() {
return _rpk_process->wait()
.then([this, job_id](auto) {
auto hold = _gate.hold();
return _process_control_mutex.get_units()
.then([this, job_id](auto units) {
return _process_control_mutex.get_units().then(
[this, job_id](auto units) {
return handle_wait_result(job_id).finally(
[units = std::move(units)] {});
})
.finally([hold = std::move(hold)] {});
});
})
.handle_exception([](const std::exception_ptr& e) {
lg.error("wait() failed while running rpk debug bundle: {}", e);
Expand Down Expand Up @@ -710,10 +718,14 @@ ss::future<> service::set_metadata(job_id_t job_id) {
}
sha256_checksum = co_await calculate_sha256_sum(
debug_bundle_file.native());
_probe->successful_bundle_generation(_rpk_process->finished_time());
} else {
_probe->failed_bundle_generation(_rpk_process->finished_time());
}

metadata md(
_rpk_process->created_time(),
_rpk_process->finished_time(),
job_id,
debug_bundle_file,
process_output_file,
Expand Down Expand Up @@ -885,6 +897,12 @@ ss::future<> service::maybe_reload_previous_run() {

_rpk_process = std::make_unique<debug_bundle_process>(
std::move(md), std::move(po));

if (run_was_successful) {
_probe->successful_bundle_generation(_rpk_process->finished_time());
} else {
_probe->failed_bundle_generation(_rpk_process->finished_time());
}
}

} // namespace debug_bundle
7 changes: 5 additions & 2 deletions src/v/debug_bundle/debug_bundle_service.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,9 @@

namespace debug_bundle {

/// Forward declare probe
class probe;

/**
* @brief Service used to manage creation of debug bundles
*
Expand All @@ -33,8 +36,6 @@ namespace debug_bundle {
*/
class service final : public ss::peering_sharded_service<service> {
public:
/// Default shard operations will be performed on
static constexpr ss::shard_id service_shard = 0;
/// Name of the debug bundle directory
static constexpr std::string_view debug_bundle_dir_name = "debug-bundle";
/// Key used to store metadata in the kvstore
Expand Down Expand Up @@ -200,6 +201,8 @@ class service final : public ss::peering_sharded_service<service> {
config::binding<std::filesystem::path> _rpk_path_binding;
/// External process
std::unique_ptr<debug_bundle_process> _rpk_process;
/// Metrics probe
std::unique_ptr<probe> _probe;
/// Mutex to guard control over the rpk debug bundle process
mutex _process_control_mutex;
ss::gate _gate;
Expand Down
12 changes: 9 additions & 3 deletions src/v/debug_bundle/metadata.h
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ struct metadata
: serde::envelope<metadata, serde::version<0>, serde::compat_version<0>> {
using time_resolution = clock::duration;
time_resolution::rep created_at{};
time_resolution::rep finished_at{};
job_id_t job_id;
ss::sstring debug_bundle_file_path;
ss::sstring process_output_file_path;
Expand All @@ -92,14 +93,14 @@ struct metadata

explicit metadata(
clock::time_point created_at,
clock::time_point finished_at,
job_id_t job_id,
const std::filesystem::path& debug_bundle_file_path,
const std::filesystem::path& process_output_file_path,
bytes sha256_checksum,
ss::experimental::process::wait_status process_result)
: created_at(std::chrono::duration_cast<time_resolution>(
created_at.time_since_epoch())
.count())
: created_at(created_at.time_since_epoch() / time_resolution{1})
, finished_at(finished_at.time_since_epoch() / time_resolution{1})
, job_id(job_id)
, debug_bundle_file_path(debug_bundle_file_path.native())
, process_output_file_path(process_output_file_path.native())
Expand All @@ -124,6 +125,7 @@ struct metadata
auto serde_fields() {
return std::tie(
created_at,
finished_at,
job_id,
debug_bundle_file_path,
process_output_file_path,
Expand All @@ -135,6 +137,10 @@ struct metadata
return clock::time_point(time_resolution(created_at));
}

clock::time_point get_finished_at() const {
return clock::time_point(time_resolution(finished_at));
}

ss::experimental::process::wait_status get_wait_status() const {
return ss::visit(
process_result,
Expand Down
81 changes: 81 additions & 0 deletions src/v/debug_bundle/probe.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* Copyright 2024 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 "debug_bundle/probe.h"

#include "config/configuration.h"
#include "debug_bundle/types.h"
#include "metrics/prometheus_sanitize.h"

#include <seastar/core/metrics.hh>
#include <seastar/core/metrics_registration.hh>
#include <seastar/core/shard_id.hh>

using namespace std::chrono_literals;

namespace debug_bundle {
void probe::setup_metrics() {
namespace sm = ss::metrics;
if (ss::this_shard_id() != service_shard) {
return;
}
auto setup_common = [this]<typename MetricDef>() {
std::vector<MetricDef> defs;
defs.reserve(4);
defs.emplace_back(
sm::make_gauge(
"last_successful_bundle_timestamp_seconds",
[this] { return _last_successful_bundle.time_since_epoch() / 1s; },
sm::description("Timestamp of last successful debug bundle "
"generation (seconds since epoch)"))
.aggregate({}));
defs.emplace_back(
sm::make_gauge(
"last_failed_bundle_timestamp_seconds",
[this] { return _last_failed_bundle.time_since_epoch() / 1s; },
sm::description("Timestamp of last failed debug bundle "
"generation (seconds since epoch)"))
.aggregate({}));
defs.emplace_back(
sm::make_counter(
"successful_generation_count",
[this] { return _successful_generation_count; },
sm::description(
"Running count of successful debug bundle generations"))
.aggregate({}));
defs.emplace_back(
sm::make_counter(
"failed_generation_count",
[this] { return _failed_generation_count; },
sm::description("Running count of failed debug bundle generations"))
.aggregate({}));

return defs;
};

auto group_name = prometheus_sanitize::metrics_name("debug_bundle");

if (!config::shard_local_cfg().disable_metrics()) {
_metrics.add_group(
group_name,
setup_common
.template operator()<ss::metrics::impl::metric_definition_impl>(),
{},
{});
}

if (!config::shard_local_cfg().disable_public_metrics()) {
_public_metrics.add_group(
group_name,
setup_common.template operator()<ss::metrics::metric_definition>());
}
}
} // namespace debug_bundle
41 changes: 41 additions & 0 deletions src/v/debug_bundle/probe.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* Copyright 2024 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 "debug_bundle/types.h"
#include "metrics/metrics.h"

namespace debug_bundle {
class probe {
public:
void setup_metrics();

void successful_bundle_generation(clock::time_point time) {
_last_successful_bundle = time;
++_successful_generation_count;
}

void failed_bundle_generation(clock::time_point time) {
_last_failed_bundle = time;
++_failed_generation_count;
}

private:
metrics::internal_metric_groups _metrics;
metrics::public_metric_groups _public_metrics;

clock::time_point _last_successful_bundle;
clock::time_point _last_failed_bundle;
uint32_t _successful_generation_count{0};
uint32_t _failed_generation_count{0};
};
} // namespace debug_bundle
4 changes: 2 additions & 2 deletions src/v/debug_bundle/tests/debug_bundle_service_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -345,14 +345,14 @@ TEST_F_CORO(debug_bundle_service_started_fixture, test_invalid_k8s_name) {

TEST_F_CORO(debug_bundle_service_started_fixture, try_running_multiple) {
auto res = co_await _service.invoke_on(
debug_bundle::service::service_shard, [](debug_bundle::service& s) {
debug_bundle::service_shard, [](debug_bundle::service& s) {
return s.initiate_rpk_debug_bundle_collection(
debug_bundle::job_id_t(uuid_t::create()), {});
});
ASSERT_TRUE_CORO(res.has_value()) << res.assume_error().message();

auto res2 = co_await _service.invoke_on(
(debug_bundle::service::service_shard + 1) % ss::smp::count,
(debug_bundle::service_shard + 1) % ss::smp::count,
[](debug_bundle::service& s) {
return s.initiate_rpk_debug_bundle_collection(
debug_bundle::job_id_t(uuid_t::create()), {});
Expand Down
11 changes: 10 additions & 1 deletion src/v/debug_bundle/tests/metadata_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,9 @@
namespace db = debug_bundle;

TEST(metadata_test, test_serialize_deserialize) {
using namespace std::chrono_literals;
auto test_time = db::clock::now();
auto stop_time = test_time + 1s;
db::job_id_t job_id{uuid_t::create()};
std::filesystem::path path{"/tmp/debug.zip"};
std::filesystem::path process_output_path{"/tmp/debug.out"};
Expand All @@ -31,7 +33,13 @@ TEST(metadata_test, test_serialize_deserialize) {
= ss::experimental::process::wait_exited{1};

db::metadata m{
test_time, job_id, path, process_output_path, test_bytes, status};
test_time,
stop_time,
job_id,
path,
process_output_path,
test_bytes,
status};

iobuf buf;
serde::write(buf, std::move(m));
Expand All @@ -46,6 +54,7 @@ TEST(metadata_test, test_serialize_deserialize) {
test_time.time_since_epoch())
.count());
EXPECT_EQ(m2.get_created_at(), test_time);
EXPECT_EQ(m2.get_finished_at(), stop_time);
EXPECT_EQ(m2.job_id, job_id);
EXPECT_EQ(m2.debug_bundle_file_path, path);
EXPECT_EQ(m2.process_output_file_path, process_output_path);
Expand Down
Loading

0 comments on commit 1442931

Please sign in to comment.