-
Notifications
You must be signed in to change notification settings - Fork 597
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #23215 from michael-redpanda/CORE-7092-Initial-deb…
…ug-bundle-service CORE-7092 initial debug bundle service
- Loading branch information
Showing
22 changed files
with
840 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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", | ||
], | ||
) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 |
Oops, something went wrong.