Skip to content

Commit

Permalink
Merge pull request #23557 from BenPope/CORE-7096-debug-bundle-handlers
Browse files Browse the repository at this point in the history
[CORE-7096] Hook admin API to debug bundle service
  • Loading branch information
michael-redpanda authored Oct 1, 2024
2 parents 4327ada + 8d3c3d9 commit 32bd075
Show file tree
Hide file tree
Showing 9 changed files with 375 additions and 25 deletions.
3 changes: 3 additions & 0 deletions src/v/debug_bundle/json.h
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,9 @@ debug_bundle::result<T> from_json(const json::Value& v) {
};

if constexpr (reflection::is_std_optional<T>) {
if (v.IsNull()) {
return std::nullopt;
}
auto r = from_json<typename T::value_type>(v);
if (r.has_value()) {
return T{std::move(r).assume_value()};
Expand Down
9 changes: 9 additions & 0 deletions src/v/debug_bundle/tests/json_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
#include <rapidjson/error/en.h>

#include <chrono>
#include <optional>
#include <type_traits>

using namespace debug_bundle;
Expand All @@ -36,11 +37,13 @@ class JsonTypeTest : public testing::Test {
};

using optional_int = std::optional<int>;
using optional_str = std::optional<ss::sstring>;
using named_int = named_type<int, struct test_named_tag>;

using JsonTestTypes = ::testing::Types<
int,
optional_int,
optional_str,
named_int,
uint64_t,
std::chrono::seconds,
Expand All @@ -66,6 +69,9 @@ TYPED_TEST(JsonTypeTest, BasicType) {
} else if constexpr (std::is_same_v<TypeParam, optional_int>) {
this->json_input = R"(42)";
this->expected = optional_int{42};
} else if constexpr (std::is_same_v<TypeParam, optional_str>) {
this->json_input = R"(null)";
this->expected = std::nullopt;
} else if constexpr (std::is_same_v<TypeParam, named_int>) {
this->json_input = R"(42)";
this->expected = named_int{42};
Expand Down Expand Up @@ -192,6 +198,9 @@ TYPED_TEST(JsonTypeTest, TypeIsInvalid) {
} else if constexpr (std::is_same_v<TypeParam, optional_int>) {
this->json_input = R"("42")";
this->expected = optional_int{42};
} else if constexpr (std::is_same_v<TypeParam, optional_str>) {
this->json_input = R"(42)";
this->expected = std::nullopt;
} else if constexpr (std::is_same_v<TypeParam, named_int>) {
this->json_input = R"("42")";
this->expected = named_int{42};
Expand Down
1 change: 1 addition & 0 deletions src/v/redpanda/admin/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,7 @@ redpanda_cc_library(
"api-doc/transaction.json.hh",
"api-doc/transform.json.hh",
"api-doc/usage.json.hh",
"debug_bundle.h",
"server.h",
"util.h",
],
Expand Down
111 changes: 96 additions & 15 deletions src/v/redpanda/admin/debug_bundle.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
#include "utils/functional.h"

#include <seastar/core/sstring.hh>
#include <seastar/http/file_handler.hh>
#include <seastar/http/reply.hh>
#include <seastar/json/json_elements.hh>
#include <seastar/util/short_streams.hh>
Expand Down Expand Up @@ -105,6 +106,26 @@ std::unique_ptr<ss::http::reply> make_error_body(

} // namespace

namespace debug_bundle {

ss::future<std::unique_ptr<ss::http::reply>> file_handler::handle(
const ss::sstring& file,
std::unique_ptr<ss::http::request> req,
std::unique_ptr<ss::http::reply> rep) {
{
auto mime_type = file.ends_with(".zip") ? "application/zip"
: "application/octet-stream";
return read(file, std::move(req), std::move(rep))
.then([mime_type](std::unique_ptr<ss::http::reply> rep) {
// read incorrectly sets the mime_type, fix it up.
rep->set_mime_type(mime_type);
return rep;
});
}
}

} // namespace debug_bundle

void admin_server::register_debug_bundle_routes() {
register_route_raw_async<superuser>(
ss::httpd::debug_bundle_json::post_debug_bundle,
Expand All @@ -127,13 +148,13 @@ void admin_server::register_debug_bundle_routes() {
std::unique_ptr<ss::http::reply> rep) {
return delete_debug_bundle(std::move(req), std::move(rep));
});
register_route_raw_async<superuser>(
register_route<superuser>(
ss::httpd::debug_bundle_json::get_debug_bundle_file,
[this](
std::unique_ptr<ss::http::request> req,
std::unique_ptr<ss::http::reply> rep) {
admin_server::request_handler_fn{[this](
std::unique_ptr<ss::http::request> req,
std::unique_ptr<ss::http::reply> rep) {
return get_debug_bundle_file(std::move(req), std::move(rep));
});
}});
register_route_raw_async<superuser>(
ss::httpd::debug_bundle_json::delete_debug_bundle_file,
[this](
Expand Down Expand Up @@ -167,16 +188,19 @@ ss::future<std::unique_ptr<ss::http::reply>> admin_server::post_debug_bundle(
std::move(job_id).assume_error(), std::move(rep));
}

auto params = from_json<debug_bundle::debug_bundle_parameters>(
obj, "config", false);
auto params
= from_json<std::optional<debug_bundle::debug_bundle_parameters>>(
obj, "config", false);
if (params.has_error()) {
co_return make_error_body(
std::move(params).assume_error(), std::move(rep));
}

auto res = co_await _debug_bundle_service.local()
.initiate_rpk_debug_bundle_collection(
job_id.assume_value(), std::move(params).assume_value());
job_id.assume_value(),
std::move(params).assume_value().value_or(
debug_bundle::debug_bundle_parameters{}));
if (res.has_error()) {
co_return make_error_body(res.assume_error(), std::move(rep));
}
Expand Down Expand Up @@ -238,24 +262,81 @@ ss::future<std::unique_ptr<ss::http::reply>> admin_server::delete_debug_bundle(
std::move(rep));
}

namespace {

debug_bundle::result<debug_bundle::job_id_t> get_debug_bundle_job_id(
debug_bundle::result<debug_bundle::debug_bundle_status_data> status_res,
const ss::sstring& filename) {
if (status_res.has_error()) {
return std::move(status_res).assume_error();
}

switch (status_res.assume_value().status) {
case debug_bundle::debug_bundle_status::running:
return debug_bundle::error_info{
debug_bundle::error_code::debug_bundle_process_running,
"Process still running"};
case debug_bundle::debug_bundle_status::error:
return debug_bundle::error_info{
debug_bundle::error_code::internal_error, "Process errored out"};
case debug_bundle::debug_bundle_status::success:
break;
}

if (status_res.assume_value().file_name != filename) {
return debug_bundle::error_info{
debug_bundle::error_code::job_id_not_recognized, "File Not Found"};
}

return status_res.assume_value().job_id;
}

} // namespace

ss::future<std::unique_ptr<ss::http::reply>>
admin_server::get_debug_bundle_file(
std::unique_ptr<ss::http::request> req,
std::unique_ptr<ss::http::reply> rep) {
auto job_id_str = req->get_path_param("filename");
co_return make_json_body(
ss::http::reply::status_type::not_implemented,
ss::json::json_void{},
std::move(rep));
auto filename = req->get_path_param("filename");

auto job_id_res = get_debug_bundle_job_id(
co_await _debug_bundle_service.local().rpk_debug_bundle_status(),
filename);
if (job_id_res.has_error()) {
co_return make_error_body(job_id_res.assume_error(), std::move(rep));
}

auto path_res = co_await _debug_bundle_service.local()
.rpk_debug_bundle_path(job_id_res.assume_value());
if (path_res.has_error()) {
co_return make_error_body(path_res.assume_error(), std::move(rep));
}

co_return co_await _debug_bundle_file_handler.local().handle(
path_res.assume_value().native(), std::move(req), std::move(rep));
}

ss::future<std::unique_ptr<ss::http::reply>>
admin_server::delete_debug_bundle_file(
std::unique_ptr<ss::http::request> req,
std::unique_ptr<ss::http::reply> rep) {
auto job_id_str = req->get_path_param("filename");
auto filename = req->get_path_param("filename");

auto job_id_res = get_debug_bundle_job_id(
co_await _debug_bundle_service.local().rpk_debug_bundle_status(),
filename);
if (job_id_res.has_error()) {
co_return make_error_body(job_id_res.assume_error(), std::move(rep));
}

auto del_res = co_await _debug_bundle_service.local()
.delete_rpk_debug_bundle(job_id_res.assume_value());
if (del_res.has_error()) {
co_return make_error_body(del_res.assume_error(), std::move(rep));
}

co_return make_json_body(
ss::http::reply::status_type::not_implemented,
ss::http::reply::status_type::no_content,
ss::json::json_void{},
std::move(rep));
}
39 changes: 39 additions & 0 deletions src/v/redpanda/admin/debug_bundle.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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 <seastar/http/file_handler.hh>

namespace debug_bundle {

/**
* A handler for reading files from the disk and streaming them to the client.
*/
class file_handler final : public ss::httpd::file_interaction_handler {
public:
file_handler() = default;

/**
* read a file from the disk and stream it in the reply.
* @param file the full path to a file on the disk
* @param req the reuest
* @param rep the reply
*/
ss::future<std::unique_ptr<ss::http::reply>> handle(
const ss::sstring& file,
std::unique_ptr<ss::http::request> req,
std::unique_ptr<ss::http::reply> rep) final;
};

} // namespace debug_bundle
5 changes: 4 additions & 1 deletion src/v/redpanda/admin/server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -335,6 +335,8 @@ admin_server::admin_server(
}

ss::future<> admin_server::start() {
co_await _debug_bundle_file_handler.start();

_blocked_reactor_notify_reset_timer.set_callback([this] {
return ss::smp::invoke_on_all([ms = _default_blocked_reactor_notify] {
ss::engine().update_blocked_reactor_notify_ms(ms);
Expand All @@ -353,7 +355,8 @@ ss::future<> admin_server::start() {

ss::future<> admin_server::stop() {
_blocked_reactor_notify_reset_timer.cancel();
return _server.stop();
co_await _server.stop();
co_await _debug_bundle_file_handler.stop();
}

void admin_server::configure_admin_routes() {
Expand Down
3 changes: 2 additions & 1 deletion src/v/redpanda/admin/server.h
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
#include "model/metadata.h"
#include "pandaproxy/rest/fwd.h"
#include "pandaproxy/schema_registry/fwd.h"
#include "redpanda/admin/debug_bundle.h"
#include "resource_mgmt/cpu_profiler.h"
#include "resource_mgmt/memory_sampling.h"
#include "rpc/connection_cache.h"
Expand All @@ -40,7 +41,6 @@
#include <seastar/core/sharded.hh>
#include <seastar/core/sstring.hh>
#include <seastar/http/exception.hh>
#include <seastar/http/file_handler.hh>
#include <seastar/http/httpd.hh>
#include <seastar/http/json_path.hh>
#include <seastar/http/request.hh>
Expand Down Expand Up @@ -739,6 +739,7 @@ class admin_server {
ss::sharded<kafka::server>& _kafka_server;
ss::sharded<cluster::tx_gateway_frontend>& _tx_gateway_frontend;
ss::sharded<debug_bundle::service>& _debug_bundle_service;
ss::sharded<debug_bundle::file_handler> _debug_bundle_file_handler;

// Value before the temporary override
std::chrono::milliseconds _default_blocked_reactor_notify;
Expand Down
Loading

0 comments on commit 32bd075

Please sign in to comment.