From ee6fd1681e7f275515d2e9b2be037fa2829db99d Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Wed, 18 Dec 2024 12:54:19 -0300 Subject: [PATCH 1/6] debug api: wrong ID for cpu_profile_shard_samples id was set to cpu_profile_sample, should be cpu_profile_shard_samples. --- src/v/redpanda/admin/api-doc/debug.json | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/v/redpanda/admin/api-doc/debug.json b/src/v/redpanda/admin/api-doc/debug.json index ac2dd9bde2d0..8b78068baea1 100644 --- a/src/v/redpanda/admin/api-doc/debug.json +++ b/src/v/redpanda/admin/api-doc/debug.json @@ -642,8 +642,8 @@ ], "models": { "cpu_profile_shard_samples": { - "id": "cpu_profile_sample", - "description": "cpu profile sample", + "id": "cpu_profile_shard_samples", + "description": "cpu profile object for one shard", "properties": { "shard_id": { "type": "long", @@ -663,7 +663,7 @@ }, "cpu_profile_sample": { "id": "cpu_profile_sample", - "description": "cpu profile sample", + "description": "an individual cpu profile sample with backtrace and count", "properties": { "user_backtrace": { "type": "string", From 45cec0c4b777b7b9cfba132922f9c685c65fbe5c Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Wed, 18 Dec 2024 16:51:04 -0300 Subject: [PATCH 2/6] remove unused headers --- src/v/redpanda/admin/debug.cc | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/v/redpanda/admin/debug.cc b/src/v/redpanda/admin/debug.cc index 79bdc4a10bf5..46bb9e62d4bc 100644 --- a/src/v/redpanda/admin/debug.cc +++ b/src/v/redpanda/admin/debug.cc @@ -12,18 +12,15 @@ #include "cluster/cloud_storage_size_reducer.h" #include "cluster/controller.h" #include "cluster/controller_stm.h" -#include "cluster/members_manager.h" #include "cluster/metadata_cache.h" #include "cluster/partition_leaders_table.h" #include "cluster/shard_table.h" #include "cluster/topics_frontend.h" #include "cluster/types.h" -#include "config/configuration.h" #include "config/node_config.h" #include "container/lw_shared_container.h" #include "json/validator.h" #include "model/fundamental.h" -#include "model/metadata.h" #include "redpanda/admin/api-doc/debug.json.hh" #include "redpanda/admin/server.h" #include "redpanda/admin/util.h" From 3a770fd6ffc3b88986897fc6a6affa07614b25b4 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Tue, 24 Dec 2024 16:19:47 -0300 Subject: [PATCH 3/6] utils: add arch library A small header which exposes the architecture of the current process as a constexpr value. We will use this in the cpu_profiler in order to record the arch in the output. Includes a (very trivial) test. --- src/v/utils/BUILD | 8 ++++++ src/v/utils/arch.h | 45 ++++++++++++++++++++++++++++++++++ src/v/utils/tests/BUILD | 13 ++++++++++ src/v/utils/tests/arch_test.cc | 24 ++++++++++++++++++ 4 files changed, 90 insertions(+) create mode 100644 src/v/utils/arch.h create mode 100644 src/v/utils/tests/arch_test.cc diff --git a/src/v/utils/BUILD b/src/v/utils/BUILD index 023f8c94c460..5803ffeadef5 100644 --- a/src/v/utils/BUILD +++ b/src/v/utils/BUILD @@ -2,6 +2,14 @@ load("//bazel:build.bzl", "redpanda_cc_library") package(default_visibility = ["//visibility:public"]) +redpanda_cc_library( + name = "arch", + hdrs = [ + "arch.h", + ], + include_prefix = "utils", +) + redpanda_cc_library( name = "auto_fmt", hdrs = [ diff --git a/src/v/utils/arch.h b/src/v/utils/arch.h new file mode 100644 index 000000000000..97bd14411261 --- /dev/null +++ b/src/v/utils/arch.h @@ -0,0 +1,45 @@ +/* + * 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 + +namespace util { +class cpu_arch { + // consteval means our string_view will have static lifetime + explicit consteval cpu_arch(std::string_view name) + : name{name} {} + + friend struct arch; + +public: + std::string_view name; + + static inline constexpr cpu_arch current(); + + bool operator<=>(const cpu_arch&) const = default; +}; + +struct arch { + static constexpr cpu_arch AMD64{"amd64"}; + static constexpr cpu_arch ARM64{"arm64"}; +}; + +inline constexpr cpu_arch cpu_arch::current() { +#if defined(__x86_64__) + return arch::AMD64; +#elif defined(__aarch64__) + return arch::ARM64 +#else +#error unknown arch +#endif +} + +} // namespace util diff --git a/src/v/utils/tests/BUILD b/src/v/utils/tests/BUILD index b8b72f887522..733f79d38ff2 100644 --- a/src/v/utils/tests/BUILD +++ b/src/v/utils/tests/BUILD @@ -534,3 +534,16 @@ redpanda_cc_gtest( "@seastar", ], ) + +redpanda_cc_gtest( + name = "arch_test", + timeout = "short", + srcs = [ + "arch_test.cc", + ], + deps = [ + "//src/v/test_utils:gtest", + "//src/v/utils:arch", + "@googletest//:gtest", + ], +) diff --git a/src/v/utils/tests/arch_test.cc b/src/v/utils/tests/arch_test.cc new file mode 100644 index 000000000000..088444c68395 --- /dev/null +++ b/src/v/utils/tests/arch_test.cc @@ -0,0 +1,24 @@ +/* + * 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 "utils/arch.h" + +#include + +using namespace util; + +#ifdef __x86_64__ +constexpr auto expected_arch = arch::AMD64; +#else +constexpr auto expected_arch = arch::ARM64; +#endif + +GTEST_TEST(arch, equality) { EXPECT_EQ(cpu_arch::current(), expected_arch); } From 5a92e4e49fdccb5a1fc0b7008d4ab5a818434249 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Tue, 24 Dec 2024 16:21:01 -0300 Subject: [PATCH 4/6] cpu_profiler: expose the sample period Expose this so we can include it in the admin API result. This can be useful to estimate how busy the reactor was, as we can calculate the utilization based on the expected number of samples (at 100% util) vs the observed number. --- src/v/resource_mgmt/cpu_profiler.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/v/resource_mgmt/cpu_profiler.h b/src/v/resource_mgmt/cpu_profiler.h index 6b0b41101519..62f12e2f5265 100644 --- a/src/v/resource_mgmt/cpu_profiler.h +++ b/src/v/resource_mgmt/cpu_profiler.h @@ -89,6 +89,10 @@ class cpu_profiler : public ss::peering_sharded_service { ss::future> collect_results_for_period( std::chrono::milliseconds timeout, std::optional shard_id); + // Return the configured sample period for the profiler. Note that this will + // return the configured value even if the profiler is currently disabled. + std::chrono::milliseconds sample_period() const { return _sample_period(); } + private: // impl for the above ss::future<> From d33997786ff7dd457117bae6a37b474c30d0e55e Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Tue, 24 Dec 2024 16:22:55 -0300 Subject: [PATCH 5/6] admin: include more info in cpu profile sample 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. --- src/v/redpanda/admin/BUILD | 2 + src/v/redpanda/admin/api-doc/debug.json | 38 ++++++++-- src/v/redpanda/admin/debug.cc | 93 ++++++++++++++++--------- 3 files changed, 97 insertions(+), 36 deletions(-) diff --git a/src/v/redpanda/admin/BUILD b/src/v/redpanda/admin/BUILD index ef9b1048a15c..dd8dd90c3b7c 100644 --- a/src/v/redpanda/admin/BUILD +++ b/src/v/redpanda/admin/BUILD @@ -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", diff --git a/src/v/redpanda/admin/api-doc/debug.json b/src/v/redpanda/admin/api-doc/debug.json index 8b78068baea1..5f6f649585f4 100644 --- a/src/v/redpanda/admin/api-doc/debug.json +++ b/src/v/redpanda/admin/api-doc/debug.json @@ -588,10 +588,7 @@ "produces": [ "application/json" ], - "type": "array", - "items": { - "type": "cpu_profile_shard_samples" - }, + "type": "cpu_profile_result", "parameters": [ { "name": "shard", @@ -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", diff --git a/src/v/redpanda/admin/debug.cc b/src/v/redpanda/admin/debug.cc index 46bb9e62d4bc..270cec2aac8c 100644 --- a/src/v/redpanda/admin/debug.cc +++ b/src/v/redpanda/admin/debug.cc @@ -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 #include @@ -502,38 +504,47 @@ void check_shard_id(seastar::shard_id id) { ss::future admin_server::cpu_profile_handler(std::unique_ptr 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 shard_id; - if (auto e = req->get_query_param("shard"); !e.empty()) { + if (!shard_param.empty()) { try { - shard_id = boost::lexical_cast(e); + shard_id = boost::lexical_cast(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 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(e)); + boost::lexical_cast(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 profiles; @@ -544,23 +555,41 @@ admin_server::cpu_profile_handler(std::unique_ptr req) { *wait_ms, shard_id); } - co_return co_await ss::make_ready_future( - 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 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 From 1793ca1f83253dcd976b53449df6bdabbd2dc8a3 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Thu, 2 Jan 2025 11:33:35 -0300 Subject: [PATCH 6/6] stress_fiber: add stack depth option Add an option to set the stack depth of the of spin loop in the stress fiber, i.e., the spin loop will at the end of a recursive call chain (not inlined) of depth N. Good for stressing CPU proflier. --- src/v/finjector/stress_fiber.cc | 28 ++++++++++++++++++++-------- src/v/finjector/stress_fiber.h | 3 +++ src/v/redpanda/admin/debug.cc | 1 + 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/src/v/finjector/stress_fiber.cc b/src/v/finjector/stress_fiber.cc index 2b96771780f1..b3200397fe82 100644 --- a/src/v/finjector/stress_fiber.cc +++ b/src/v/finjector/stress_fiber.cc @@ -34,7 +34,7 @@ class stress_payload { // Runs a single fiber that spins the given number of times per scheduling // point, until an abort is requested. - ss::future<> run_count_fiber(int min_count, int max_count); + ss::future<> run_count_fiber(int depth, int min_count, int max_count); ss::future<> run_delay_fiber(int min_ms, int max_ms); ss::gate _gate; @@ -48,6 +48,7 @@ stress_payload::stress_payload(stress_config cfg) { for (size_t i = 0; i < cfg.num_fibers; i++) { ssx::spawn_with_gate(_gate, [cfg, this] { return run_count_fiber( + cfg.stack_depth.value_or(0), *cfg.min_spins_per_scheduling_point, *cfg.max_spins_per_scheduling_point); }); @@ -66,13 +67,11 @@ stress_payload::stress_payload(stress_config cfg) { } } -ss::future<> stress_payload::run_count_fiber(int min_count, int max_count) { - while (!_as.abort_requested()) { - int spins_per_scheduling_point = min_count == max_count - ? min_count - : random_generators::get_int( - min_count, max_count); - co_await ss::maybe_yield(); +[[gnu::noinline]] +static void spinner(int depth, int spins_per_scheduling_point) { // NOLINT + if (depth > 0) { + spinner(depth - 1, spins_per_scheduling_point); + } else { volatile int spins = 0; while (true) { if (spins == spins_per_scheduling_point) { @@ -83,6 +82,19 @@ ss::future<> stress_payload::run_count_fiber(int min_count, int max_count) { } } +ss::future<> +// NOLINTNEXTLINE +stress_payload::run_count_fiber(int depth, int min_count, int max_count) { + while (!_as.abort_requested()) { + int spins_per_scheduling_point = min_count == max_count + ? min_count + : random_generators::get_int( + min_count, max_count); + spinner(depth, spins_per_scheduling_point); + co_await ss::maybe_yield(); + } +} + ss::future<> stress_payload::run_delay_fiber(int min_ms, int max_ms) { while (!_as.abort_requested()) { int ms_per_scheduling_point = min_ms == max_ms diff --git a/src/v/finjector/stress_fiber.h b/src/v/finjector/stress_fiber.h index c97198db616e..5528fff5972e 100644 --- a/src/v/finjector/stress_fiber.h +++ b/src/v/finjector/stress_fiber.h @@ -21,6 +21,9 @@ struct stress_config { // If set, "delay" variants should not be set. std::optional min_spins_per_scheduling_point; std::optional max_spins_per_scheduling_point; + // The (additional) stack depth the spin function should + // have, which is useful for stressing the CPU profiler. + std::optional stack_depth; // Time in milliseconds to spin for between each scheduling point. // If set, "spins" variants should not be set. diff --git a/src/v/redpanda/admin/debug.cc b/src/v/redpanda/admin/debug.cc index 270cec2aac8c..bd8ae7733fe7 100644 --- a/src/v/redpanda/admin/debug.cc +++ b/src/v/redpanda/admin/debug.cc @@ -148,6 +148,7 @@ void admin_server::register_debug_routes() { parse_int( "max_spins_per_scheduling_point", cfg.max_spins_per_scheduling_point); + parse_int("stack_depth", cfg.stack_depth); parse_int( "min_ms_per_scheduling_point", cfg.min_ms_per_scheduling_point); parse_int(