Skip to content

Commit

Permalink
admin: include more info in cpu profile sample
Browse files Browse the repository at this point in the history
Change the output format of the cpu profiler API to include:

 - The CPU architecture
 - The version string
 - The wait_ms, if specified
 - The profiler sample period
 - The schema version of the API response

The first two of the above enable us to symbolize profiles directly from
the result without needing to know the version/arch and download
symbols separately.
  • Loading branch information
travisdowns committed Dec 24, 2024
1 parent 5a92e4e commit d339977
Show file tree
Hide file tree
Showing 3 changed files with 97 additions and 36 deletions.
2 changes: 2 additions & 0 deletions src/v/redpanda/admin/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -247,8 +247,10 @@ redpanda_cc_library(
"//src/v/strings:utf8",
"//src/v/transform",
"//src/v/transform:fwd",
"//src/v/utils:arch",
"//src/v/utils:functional",
"//src/v/utils:unresolved_address",
"//src/v/version",
"//src/v/wasm:api",
"@abseil-cpp//absl/container:flat_hash_map",
"@abseil-cpp//absl/container:flat_hash_set",
Expand Down
38 changes: 34 additions & 4 deletions src/v/redpanda/admin/api-doc/debug.json
Original file line number Diff line number Diff line change
Expand Up @@ -588,10 +588,7 @@
"produces": [
"application/json"
],
"type": "array",
"items": {
"type": "cpu_profile_shard_samples"
},
"type": "cpu_profile_result",
"parameters": [
{
"name": "shard",
Expand Down Expand Up @@ -641,6 +638,39 @@
}
],
"models": {
"cpu_profile_result": {
"id": "cpu_profile_result",
"description": "top level object for a cpu profile request",
"properties" : {
"schema": {
"description": "the schema version of the response",
"type": "int"
},
"arch": {
"description": "the CPU architecture the profile was taken on, one of [amd64, arm64]",
"type": "string"
},
"version": {
"description": "the version of Redpanda the profile was taken on",
"type": "string"
},
"wait_ms": {
"description": "the requested sample period, in milliseconds, if specified using wait_ms query parameter, or missing otherwise",
"type": "int"
},
"sample_period_ms": {
"description": "the configured sample period in milliseconds (each shard samples at this rate)",
"type": "int"
},
"profile": {
"description": "the profile samples, one object per shard",
"type": "array",
"items": {
"type": "cpu_profile_shard_samples"
}
}
}
},
"cpu_profile_shard_samples": {
"id": "cpu_profile_shard_samples",
"description": "cpu profile object for one shard",
Expand Down
93 changes: 61 additions & 32 deletions src/v/redpanda/admin/debug.cc
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
#include "resource_mgmt/cpu_profiler.h"
#include "serde/rw/rw.h"
#include "storage/kvstore.h"
#include "utils/arch.h"
#include "version/version.h"

#include <seastar/core/shard_id.hh>
#include <seastar/core/sstring.hh>
Expand Down Expand Up @@ -502,38 +504,47 @@ void check_shard_id(seastar::shard_id id) {

ss::future<ss::json::json_return_type>
admin_server::cpu_profile_handler(std::unique_ptr<ss::http::request> req) {
vlog(adminlog.info, "Request to sampled cpu profile");
using namespace ss::httpd::debug_json;

auto shard_param = req->get_query_param("shard");
auto wait_param = req->get_query_param("wait_ms");

vlog(
adminlog.info,
"Request to sample cpu profile, shard: {}, wait_ms: {}",
shard_param,
wait_param);

cpu_profile_result result;
// update this when you make an incompatible change to the result schema
result.schema = 2;
result.sample_period_ms = _cpu_profiler.local().sample_period() / 1ms;

std::optional<size_t> shard_id;
if (auto e = req->get_query_param("shard"); !e.empty()) {
if (!shard_param.empty()) {
try {
shard_id = boost::lexical_cast<size_t>(e);
shard_id = boost::lexical_cast<size_t>(shard_param);
} catch (const boost::bad_lexical_cast&) {
throw ss::httpd::bad_param_exception(
fmt::format("Invalid parameter 'shard_id' value {{{}}}", e));
throw ss::httpd::bad_param_exception(fmt::format(
"Invalid parameter 'shard_id' value {{{}}}", shard_param));
}
}

if (shard_id.has_value()) {
check_shard_id(*shard_id);
}

std::optional<std::chrono::milliseconds> wait_ms;
if (auto e = req->get_query_param("wait_ms"); !e.empty()) {
if (!wait_param.empty()) {
try {
wait_ms = std::chrono::milliseconds(
boost::lexical_cast<uint64_t>(e));
boost::lexical_cast<uint64_t>(wait_param));
} catch (const boost::bad_lexical_cast&) {
throw ss::httpd::bad_param_exception(
fmt::format("Invalid parameter 'wait_ms' value {{{}}}", e));
throw ss::httpd::bad_param_exception(fmt::format(
"Invalid parameter 'wait_ms' value {{{}}}", wait_param));
}
}

if (wait_ms.has_value()) {
if (*wait_ms < 1ms || *wait_ms > 15min) {
throw ss::httpd::bad_param_exception(
"wait_ms must be between 1ms and 15min");
}
result.wait_ms = *wait_ms / 1ms;
}

std::vector<resources::cpu_profiler::shard_samples> profiles;
Expand All @@ -544,23 +555,41 @@ admin_server::cpu_profile_handler(std::unique_ptr<ss::http::request> req) {
*wait_ms, shard_id);
}

co_return co_await ss::make_ready_future<ss::json::json_return_type>(
ss::json::stream_range_as_array(
lw_shared_container(std::move(profiles)),
[](const resources::cpu_profiler::shard_samples& profile) {
ss::httpd::debug_json::cpu_profile_shard_samples ret;
ret.shard_id = profile.shard;
ret.dropped_samples = profile.dropped_samples;

for (auto& sample : profile.samples) {
ss::httpd::debug_json::cpu_profile_sample s;
s.occurrences = sample.occurrences;
s.user_backtrace = sample.user_backtrace;

ret.samples.push(s);
}
return ret;
}));
result.arch = ss::sstring{util::cpu_arch::current().name};
// this version will help us identify the right symbols, it is like so:
// In released builds:
// v24.2.11 - 29b8a8e2329043d587e6de2cbf8e73cc32d9d69e
// Local bazel builds:
// 0.0.0-dev - 0000000000000000000000000000000000000000
// Local cmake builds with ENABLE_GIT_HASH=OFF and ENABLE_GIT_VERSION=OFF:
// no_version - 000-dev
result.version = ss::sstring{redpanda_version()};

auto& profile_vec = result.profile._elements;
profile_vec.reserve(profiles.size());

for (auto& shard_profile : profiles) {
ss::httpd::debug_json::cpu_profile_shard_samples shard_samples;
shard_samples.shard_id = shard_profile.shard;
shard_samples.dropped_samples = shard_profile.dropped_samples;

// build up the samples list
std::vector<cpu_profile_sample> samples;
samples.reserve(shard_profile.samples.size());
for (auto& sample : shard_profile.samples) {
ss::httpd::debug_json::cpu_profile_sample json_sample;
json_sample.occurrences = sample.occurrences;
json_sample.user_backtrace = sample.user_backtrace;
samples.emplace_back(std::move(json_sample));
}

shard_samples.samples._set = true;
shard_samples.samples._elements = std::move(samples);

result.profile.push(shard_samples);
}

co_return co_await ssx::now(stream_object(result));
}

ss::future<ss::json::json_return_type>
Expand Down

0 comments on commit d339977

Please sign in to comment.