Skip to content

Commit

Permalink
test_utils: add profile_helper to get profiles in tests
Browse files Browse the repository at this point in the history
  • Loading branch information
ballard26 committed Nov 20, 2024
1 parent a262de2 commit 3712bfa
Show file tree
Hide file tree
Showing 3 changed files with 203 additions and 0 deletions.
9 changes: 9 additions & 0 deletions src/v/test_utils/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -22,3 +22,12 @@ v_cc_library(

add_subdirectory(go/kreq-gen)
add_subdirectory(tests)

v_cc_library(
NAME profile_helper
SRCS
profile_helper.cc
DEPS
Seastar::seastar
v::resource_mgmt
)
147 changes: 147 additions & 0 deletions src/v/test_utils/profile_helper.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
/*
* 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_helper.h"

#include "container/lw_shared_container.h"
#include "redpanda/admin/api-doc/debug.json.hh"

#include <seastar/core/file.hh>
#include <seastar/core/fstream.hh>
#include <seastar/core/seastar.hh>
#include <seastar/http/httpd.hh>
#include <seastar/json/json_elements.hh>

#include <limits>
#include <optional>

ss::future<> profile_helper::start() {
co_await _cpu_profiler.start(
config::mock_binding(true),
config::mock_binding<std::chrono::milliseconds>(_sample_period));
co_await _cpu_profiler.invoke_on_all(&resources::cpu_profiler::start);
co_await _memory_sampler.start(
std::ref(_logger), config::mock_binding<bool>(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<ss::file> 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<resources::cpu_profiler::shard_samples> profiles
= co_await _cpu_profiler.local().results(std::nullopt);

auto json_res = 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;
}));

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<ss::httpd::debug_json::memory_profile> 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";

ss::future<size_t> next_free_profile_file_id(std::filesystem::path path) {
for (size_t i = 0; i < std::numeric_limits<size_t>::max(); i++) {
auto cpu_p_exists = co_await ss::file_exists(
(path / std::format(cpu_profile_file_template, i)).string());
auto memory_p_exists = co_await ss::file_exists(
(path / std::format(memory_profile_file_template, i)).string());
if (!cpu_p_exists && !memory_p_exists) {
co_return i;
}
}

co_return 0;
}
} // namespace

ss::future<> profile_section(
std::function<ss::future<>()> section, std::filesystem::path output_dir) {
auto profile_id = co_await next_free_profile_file_id(output_dir);
auto cpu_profile_path = output_dir
/ std::format(
cpu_profile_file_template, profile_id);
auto memory_profile_path = output_dir
/ std::format(
memory_profile_file_template, profile_id);

profile_helper ph(std::chrono::milliseconds(5));
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();
}
47 changes: 47 additions & 0 deletions src/v/test_utils/profile_helper.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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 "resource_mgmt/cpu_profiler.h"
#include "resource_mgmt/memory_sampling.h"

#include <seastar/core/sharded.hh>
#include <seastar/util/log.hh>

#include <chrono>
#include <functional>

/**
* \brief `profile_helper` is designed to be used in unit/micorbenchmark tests
* to enable collecting cpu/memory profiles during the test.
*/
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<resources::cpu_profiler> _cpu_profiler;
ss::sharded<memory_sampling> _memory_sampler;
seastar::logger _logger{"profile_logger"};
};

ss::future<> profile_section(
std::function<ss::future<>()> section, std::filesystem::path output_dir);

0 comments on commit 3712bfa

Please sign in to comment.