Skip to content

Commit

Permalink
Merge pull request #23215 from michael-redpanda/CORE-7092-Initial-deb…
Browse files Browse the repository at this point in the history
…ug-bundle-service

CORE-7092 initial debug bundle service
  • Loading branch information
michael-redpanda authored Sep 13, 2024
2 parents 80d6263 + 99d95a2 commit 3bf5869
Show file tree
Hide file tree
Showing 22 changed files with 840 additions and 0 deletions.
1 change: 1 addition & 0 deletions src/v/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@ add_subdirectory(wasm)
add_subdirectory(transform)
add_subdirectory(net)
add_subdirectory(rpc)
add_subdirectory(debug_bundle)

# libraries
add_subdirectory(test_utils)
Expand Down
6 changes: 6 additions & 0 deletions src/v/config/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3267,6 +3267,12 @@ configuration::configuration()
{.needs_restart = needs_restart::no, .visibility = visibility::user},
100ms,
{.min = 1ms})
, rpk_path(
*this,
"rpk_path",
"Path to RPK binary",
{.needs_restart = needs_restart::no, .visibility = visibility::tunable},
"/usr/bin/rpk")
, oidc_discovery_url(
*this,
"oidc_discovery_url",
Expand Down
1 change: 1 addition & 0 deletions src/v/config/configuration.h
Original file line number Diff line number Diff line change
Expand Up @@ -601,6 +601,7 @@ struct configuration final : public config_store {
// debug controls
property<bool> cpu_profiler_enabled;
bounded_property<std::chrono::milliseconds> cpu_profiler_sample_period_ms;
property<std::filesystem::path> rpk_path;

// oidc authentication
property<ss::sstring> oidc_discovery_url;
Expand Down
32 changes: 32 additions & 0 deletions src/v/debug_bundle/BUILD
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
load("//bazel:build.bzl", "redpanda_cc_library")

redpanda_cc_library(
name = "debug_bundle",
srcs = [
"debug_bundle_service.cc",
"error.cc",
"types.cc",
],
hdrs = [
"debug_bundle_service.h",
"error.h",
"fwd.h",
"types.h",
],
implementation_deps = [
"//src/v/strings:string_switch",
],
include_prefix = "debug_bundle",
visibility = ["//visibility:public"],
deps = [
"//src/v/base",
"//src/v/config",
"//src/v/container:fragmented_vector",
"//src/v/model",
"//src/v/security",
"//src/v/utils:uuid",
"@abseil-cpp//absl/container:btree",
"@fmt",
"@seastar",
],
)
12 changes: 12 additions & 0 deletions src/v/debug_bundle/CMakeLists.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
v_cc_library(
NAME debug_bundle
SRCS
debug_bundle_service.cc
error.cc
types.cc
DEPS
v::config
v::utils
)

add_subdirectory(tests)
106 changes: 106 additions & 0 deletions src/v/debug_bundle/debug_bundle_service.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
* 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_service.h"

#include "config/configuration.h"
#include "debug_bundle/error.h"
#include "debug_bundle/types.h"

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

namespace debug_bundle {
static ss::logger lg{"debug-bundle-service"};

service::service(const std::filesystem::path& data_dir)
: _debug_bundle_dir(data_dir / debug_bundle_dir_name)
, _rpk_path_binding(config::shard_local_cfg().rpk_path.bind()) {}

ss::future<> service::start() {
if (ss::this_shard_id() != service_shard) {
co_return;
}

try {
lg.trace("Creating {}", _debug_bundle_dir);
co_await ss::recursive_touch_directory(_debug_bundle_dir.native());
} catch (const std::exception& e) {
throw std::system_error(error_code::internal_error, e.what());
}

if (!co_await ss::file_exists(_rpk_path_binding().native())) {
lg.error(
"Current specified RPK location {} does not exist! Debug "
"bundle creation is not available until this is fixed!",
_rpk_path_binding().native());
}

lg.debug("Service started");
}

ss::future<> service::stop() {
if (ss::this_shard_id() != service_shard) {
co_return;
}
lg.debug("Service stopping");
}

ss::future<result<void>> service::initiate_rpk_debug_bundle_collection(
uuid_t uuid, debug_bundle_parameters params) {
if (ss::this_shard_id() != service_shard) {
co_return co_await container().invoke_on(
service_shard,
[uuid, params = std::move(params)](service& s) mutable {
return s.initiate_rpk_debug_bundle_collection(
uuid, std::move(params));
});
}
co_return error_info(error_code::internal_error, "Not yet implemented");
}

ss::future<result<void>> service::cancel_rpk_debug_bundle(uuid_t uuid) {
if (ss::this_shard_id() != service_shard) {
co_return co_await container().invoke_on(
service_shard,
[uuid](service& s) { return s.cancel_rpk_debug_bundle(uuid); });
}
co_return error_info(error_code::debug_bundle_process_never_started);
}

ss::future<result<debug_bundle_status_data>>
service::rpk_debug_bundle_status() {
if (ss::this_shard_id() != service_shard) {
co_return co_await container().invoke_on(service_shard, [](service& s) {
return s.rpk_debug_bundle_status();
});
}
co_return error_info(error_code::debug_bundle_process_never_started);
}

ss::future<result<std::filesystem::path>> service::rpk_debug_bundle_path() {
if (ss::this_shard_id() != service_shard) {
co_return co_await container().invoke_on(
service_shard, [](service& s) { return s.rpk_debug_bundle_path(); });
}
co_return error_info(error_code::debug_bundle_process_never_started);
}

ss::future<result<void>> service::delete_rpk_debug_bundle() {
if (ss::this_shard_id() != service_shard) {
co_return co_await container().invoke_on(service_shard, [](service& s) {
return s.delete_rpk_debug_bundle();
});
}
co_return error_info(error_code::debug_bundle_process_never_started);
}

} // namespace debug_bundle
114 changes: 114 additions & 0 deletions src/v/debug_bundle/debug_bundle_service.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/*
* 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 "base/seastarx.h"
#include "config/property.h"
#include "debug_bundle/error.h"
#include "debug_bundle/types.h"
#include "utils/uuid.h"

#include <seastar/core/sharded.hh>

namespace debug_bundle {

/**
* @brief Service used to manage creation of debug bundles
*
* This service is used to create, delete, and manage debug bundles using the
* "rpk debug bundle" application
*/
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";
/**
* @brief Construct a new debug bundle service object
*
* @param data_dir Path to the Redpanda data directory
*/
explicit service(const std::filesystem::path& data_dir);
/**
* @brief Starts the service
*
* Starting the service will:
* * Create the debug bundle directory
* * Verify that the rpk binary is present
*/
ss::future<> start();
/**
* @brief Halts the service
*/
ss::future<> stop();

/**
* @brief Initializes the creation of a debug bundle
*
* @param job_id The job ID
* @param params the parameters
* @return Result with possible error codes:
* * error_code::debug_bundle_process_running
* * error_code::invalid_parameters
* * error_code::process_failed
* * error_code::internal_error
*/
ss::future<result<void>> initiate_rpk_debug_bundle_collection(
uuid_t job_id, debug_bundle_parameters params);

/**
* @brief Attempts to cancel a running "rpk debug bundle" process
*
* @param job_id The Job ID to cancel
* @return ss::future<result<void>> The result with possible error codes:
* * error_code::debug_bundle_process_not_running
* * error_code::internal_error
* * error_code::job_id_not_recognized
* * error_code::debug_bundle_process_never_started
*/
ss::future<result<void>> cancel_rpk_debug_bundle(uuid_t job_id);

/**
* @brief Retrieves the status of the bundle process
*
* @return ss::future<result<debug_bundle_status_data>> The result with
* possible error codes:
* * error_code::debug_bundle_process_never_started
*/
ss::future<result<debug_bundle_status_data>> rpk_debug_bundle_status();

/**
* @brief Returns the path to the debug bundle file
*
* @return ss::future<result<std::filesystem::path>> The result with
* possible error codes:
* * error_code::debug_bundle_process_never_started
*/
ss::future<result<std::filesystem::path>> rpk_debug_bundle_path();

/**
* @brief Attempts to delete the debug bundle file
*
* @return ss::future<result<void>> The result with possible error codes:
* * error_code::debug_bundle_process_never_started
* * error_code::internal_error
*/
ss::future<result<void>> delete_rpk_debug_bundle();

private:
/// Path to the debug bundle directory
std::filesystem::path _debug_bundle_dir;
/// Binding called when the rpk path config changes
config::binding<std::filesystem::path> _rpk_path_binding;
};
} // namespace debug_bundle
52 changes: 52 additions & 0 deletions src/v/debug_bundle/error.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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 "error.h"

#include <system_error>

namespace debug_bundle {
namespace {
struct error_category final : std::error_category {
const char* name() const noexcept override { return "debug_bundle"; }

std::string message(int ev) const override {
switch (static_cast<error_code>(ev)) {
case error_code::success:
return "success";
case error_code::debug_bundle_process_running:
return "debug bundle process currently running";
case error_code::debug_bundle_process_not_running:
return "debug bundle process not currently running";
case error_code::invalid_parameters:
return "invalid parameters";
case error_code::process_failed:
return "process failed";
case error_code::internal_error:
return "internal system error";
case error_code::job_id_not_recognized:
return "job id not recognized";
case error_code::debug_bundle_process_never_started:
return "debug bundle process was never started";
}

return "(unknown error code)";
}
};

const error_category debug_bundle_error_category{};
} // namespace

std::error_code make_error_code(error_code e) noexcept {
return {static_cast<int>(e), debug_bundle_error_category};
}

} // namespace debug_bundle
Loading

0 comments on commit 3bf5869

Please sign in to comment.