diff --git a/src/v/test_utils/BUILD b/src/v/test_utils/BUILD index 39f63317f9a53..a991724f0ea6c 100644 --- a/src/v/test_utils/BUILD +++ b/src/v/test_utils/BUILD @@ -128,3 +128,24 @@ redpanda_test_cc_library( "@seastar", ], ) + +redpanda_test_cc_library( + name = "profile_utils", + srcs = [ + "profile_utils.cc", + ], + hdrs = [ + "profile_utils.h", + ], + implementation_deps = [ + "//src/v/redpanda/admin", + "//src/v/resource_mgmt:cpu_profiler", + "//src/v/resource_mgmt:memory_sampling", + ], + include_prefix = "test_utils", + visibility = ["//visibility:public"], + deps = [ + "//src/v/base", + "@seastar", + ], +) diff --git a/src/v/test_utils/CMakeLists.txt b/src/v/test_utils/CMakeLists.txt index 4f88aa657ed91..0655e519f3bd1 100644 --- a/src/v/test_utils/CMakeLists.txt +++ b/src/v/test_utils/CMakeLists.txt @@ -22,3 +22,12 @@ v_cc_library( add_subdirectory(go/kreq-gen) add_subdirectory(tests) + +v_cc_library( + NAME profile_utils + SRCS + profile_utils.cc + DEPS + Seastar::seastar + v::resource_mgmt +) diff --git a/src/v/test_utils/profile_utils.cc b/src/v/test_utils/profile_utils.cc new file mode 100644 index 0000000000000..853ff6784909f --- /dev/null +++ b/src/v/test_utils/profile_utils.cc @@ -0,0 +1,215 @@ +/* + * 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 "test_utils/profile_utils.h" + +#include "container/lw_shared_container.h" +#include "redpanda/admin/api-doc/debug.json.hh" +#include "resource_mgmt/cpu_profiler.h" +#include "resource_mgmt/memory_sampling.h" + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace test_utils { + +class profile_helper { +public: + explicit profile_helper( + std::chrono::milliseconds cpu_profiler_sample_period) + : _sample_period(cpu_profiler_sample_period) {} + + ss::future<> start(); + ss::future<> stop(); + + ss::future<> write_cpu_profiler_results(std::filesystem::path path); + ss::future<> write_memory_profiler_results(std::filesystem::path path); + +private: + std::chrono::milliseconds _sample_period; + ss::sharded _cpu_profiler; + ss::sharded _memory_sampler; + seastar::logger _logger{"profile_helper_logger"}; +}; + +ss::future<> profile_helper::start() { + co_await _cpu_profiler.start( + config::mock_binding(true), + config::mock_binding(_sample_period)); + co_await _cpu_profiler.invoke_on_all(&resources::cpu_profiler::start); + co_await _memory_sampler.start( + std::ref(_logger), config::mock_binding(true)); + co_await _memory_sampler.invoke_on_all(&memory_sampling::start); +} + +ss::future<> profile_helper::stop() { + co_await _cpu_profiler.stop(); + co_await _memory_sampler.stop(); +} + +namespace { +ss::future make_handle( + std::filesystem::path path, ss::open_flags flags, ss::file_open_options opt) { + auto file = co_await ss::open_file_dma(path.string(), flags, opt); + + co_return std::move(file); +} + +ss::future<> write_to_file( + std::filesystem::path path, seastar::json::json_return_type json) { + auto handle = co_await make_handle( + std::move(path), ss::open_flags::create | ss::open_flags::wo, {}); + auto ostream = co_await ss::make_file_output_stream(std::move(handle)); + if (json._body_writer) { + co_await json._body_writer(std::move(ostream)); + } else { + co_await ostream.write(json._res); + co_await ostream.close(); + } +} +} // namespace + +ss::future<> +profile_helper::write_cpu_profiler_results(std::filesystem::path path) { + std::vector profiles + = co_await _cpu_profiler.local().results(std::nullopt); + + auto json_res = 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; + })); + + co_await write_to_file(std::move(path), std::move(json_res)); +} + +ss::future<> +profile_helper::write_memory_profiler_results(std::filesystem::path path) { + auto profiles = co_await _memory_sampler.local() + .get_sampled_memory_profiles(std::nullopt); + + std::vector resp(profiles.size()); + for (size_t i = 0; i < resp.size(); ++i) { + resp[i].shard = profiles[i].shard_id; + + for (auto& allocation_sites : profiles[i].allocation_sites) { + ss::httpd::debug_json::allocation_site allocation_site; + allocation_site.size = allocation_sites.size; + allocation_site.count = allocation_sites.count; + allocation_site.backtrace = std::move(allocation_sites.backtrace); + resp[i].allocation_sites.push(allocation_site); + } + } + + co_await write_to_file(std::move(path), resp); +} + +namespace { +constexpr auto cpu_profile_file_template = "{}_cpu_profile_{}.json"; +constexpr auto memory_profile_file_template = "{}_memory_profile_{}.json"; + +std::string fmt_cpu_prof(std::string_view section_name, size_t id) { + return std::format(cpu_profile_file_template, section_name, id); +} + +std::string fmt_mem_prof(std::string_view section_name, size_t id) { + return std::format(memory_profile_file_template, section_name, id); +} + +ss::future profiles_exists_for( + std::string_view section_name, + std::filesystem::path path, + size_t profile_id) { + auto cpu_p_exists = co_await ss::file_exists( + (path / fmt_cpu_prof(section_name, profile_id)).string()); + auto memory_p_exists = co_await ss::file_exists( + (path / fmt_mem_prof(section_name, profile_id)).string()); + co_return cpu_p_exists || memory_p_exists; +} + +ss::future next_free_profile_file_id( + std::string_view section_name, std::filesystem::path path) { + for (size_t i = 0; i < std::numeric_limits::max(); i++) { + auto profiles_exist = co_await profiles_exists_for( + section_name, path, i); + if (!profiles_exist) { + co_return i; + } + } + + co_return 0; +} +} // namespace + +ss::future<> profile_section( + std::string_view section_name, + std::function()> section, + profile_options opts) { + static std::mutex mux{}; + bool skip_profiles = false; + + std::unique_lock lock(mux, std::defer_lock); + if (!lock.try_lock()) { + // If some other section is concurrently being profiled then skip + // profiling this section. + co_await section(); + co_return; + } + + if (opts.once) { + skip_profiles = co_await profiles_exists_for( + section_name, opts.output_dir, 0); + } + + if (skip_profiles) { + co_await section(); + co_return; + } + + auto profile_id = co_await next_free_profile_file_id( + section_name, opts.output_dir); + auto cpu_profile_path = opts.output_dir + / fmt_cpu_prof(section_name, profile_id); + auto memory_profile_path = opts.output_dir + / fmt_mem_prof(section_name, profile_id); + + profile_helper ph(opts.profile_sample_period); + co_await ph.start(); + + co_await section(); + + co_await ph.write_cpu_profiler_results(cpu_profile_path); + co_await ph.write_memory_profiler_results(memory_profile_path); + co_await ph.stop(); +} + +} // namespace test_utils diff --git a/src/v/test_utils/profile_utils.h b/src/v/test_utils/profile_utils.h new file mode 100644 index 0000000000000..4abb98935a5a3 --- /dev/null +++ b/src/v/test_utils/profile_utils.h @@ -0,0 +1,75 @@ +/* + * 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 + +#include +#include +#include + +namespace test_utils { +/** + * Profile Options + */ +struct profile_options { + /// For a given section only collect a one profile regardless of how many + /// times the section runs. + bool once{false}; + /// The sample period for the CPU profiler. + std::chrono::milliseconds profile_sample_period{5}; + /// The directory the profiles should be written to. + std::filesystem::path output_dir{std::filesystem::current_path()}; +}; + +/** + * Enables profilers for the duration that \p section runs. Writes resulting + * profiles out to files named using \p section_name. + * + * This function is intended to only be used in tests. + * + * Usage + * ===== + * + * Consider the following section of code from a microbenchmark; + * + * perf_tests::start_measuring_time(); + * auto res = co_await reader.consume( + * std::move(consumer), model::no_timeout); + * perf_tests::stop_measuring_time(); + * + * To get profiles for the section `profile_section` would be used; + * + * co_await profile_section("example_section_name", [&]() -> ss::future<> { + * perf_tests::start_measuring_time(); + * auto res = co_await reader.consume( + * std::move(consumer), model::no_timeout); + * perf_tests::stop_measuring_time(); + * }); + * + * This will result in the following files in the current working directory; + * - example_section_name_cpu_profile_N.json + * - example_section_name_memory_profile_N.json + * + * Where `N` indicates which run of the section the profile is for, with 0 + * indicating the first run. + * + * Note that this function will not override any existing files. + */ +ss::future<> profile_section( + std::string_view section_name, + std::function()> section, + profile_options opts = {}); + +} // namespace test_utils