Skip to content

Commit

Permalink
config: Add config flag for memory profiling
Browse files Browse the repository at this point in the history
Adds a config flag to enable/disable the memory sampling.

Flag can be changed without a restart. However, turning it on after the
fact is not that useful because we will be missing lots of live
allocations.

It's still useful if profiling needs to be turned off for whatever
reason.
  • Loading branch information
StephanDollberg committed Jun 14, 2023
1 parent 4306a0d commit 6c0f1c0
Show file tree
Hide file tree
Showing 18 changed files with 112 additions and 27 deletions.
13 changes: 13 additions & 0 deletions src/v/config/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1881,6 +1881,19 @@ configuration::configuration()
"exception is thrown instead.",
{.needs_restart = needs_restart::no, .visibility = visibility::tunable},
true)
, sampled_memory_profile(
*this,
"memory_enable_memory_sampling",
"If true, memory allocations will be sampled and tracked. A sampled live "
"set of allocations can then be retrieved from the Admin API. "
"Additionally, we will periodically log the top-n allocation sites",
{// Enabling/Disabling this dynamically doesn't make much sense as for the
// memory profile to be meaning full you'll want to have this on from the
// beginning. However, we still provide the option to be able to disable
// it dynamically in case something goes wrong
.needs_restart = needs_restart::no,
.visibility = visibility::tunable},
true)
, enable_metrics_reporter(
*this,
"enable_metrics_reporter",
Expand Down
1 change: 1 addition & 0 deletions src/v/config/configuration.h
Original file line number Diff line number Diff line change
Expand Up @@ -375,6 +375,7 @@ struct configuration final : public config_store {

// memory related settings
property<bool> memory_abort_on_alloc_failure;
property<bool> sampled_memory_profile;

// metrics reporter
property<bool> enable_metrics_reporter;
Expand Down
4 changes: 3 additions & 1 deletion src/v/raft/tests/bootstrap_configuration_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,9 @@ struct bootstrap_fixture : raft::simple_record_fixture {
.invoke_on_all(
[](features::feature_table& f) { f.testing_activate_all(); })
.get();
_memory_sampling_service.start(std::ref(_test_logger)).get();
_memory_sampling_service
.start(std::ref(_test_logger), config::mock_binding<bool>(false))
.get();
_storage.start().get();
// ignore the get_log()
(void)_storage.log_mgr()
Expand Down
4 changes: 3 additions & 1 deletion src/v/raft/tests/configuration_manager_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,9 @@ struct config_manager_fixture {
.invoke_on_all(
[](features::feature_table& f) { f.testing_activate_all(); })
.get();
_memory_sampling_service.start(std::ref(_test_logger)).get();
_memory_sampling_service
.start(std::ref(_test_logger), config::mock_binding<bool>(false))
.get();
_storage.start().get0();
}

Expand Down
4 changes: 3 additions & 1 deletion src/v/raft/tests/foreign_entry_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,9 @@ struct foreign_entry_fixture {
.invoke_on_all(
[](features::feature_table& f) { f.testing_activate_all(); })
.get();
_memory_sampling_service.start(std::ref(_test_logger)).get();
_memory_sampling_service
.start(std::ref(_test_logger), config::mock_binding<bool>(false))
.get();
_storage.start().get();
(void)_storage.log_mgr()
.manage(storage::ntp_config(_ntp, "test.dir"))
Expand Down
4 changes: 3 additions & 1 deletion src/v/raft/tests/mux_state_machine_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,9 @@ struct mux_state_machine_fixture {
[](features::feature_table& f) { f.testing_activate_all(); })
.get();

_memory_sampling_service.start(std::ref(_test_logger)).get();
_memory_sampling_service
.start(std::ref(_test_logger), config::mock_binding<bool>(false))
.get();

_group_mgr
.start(
Expand Down
4 changes: 3 additions & 1 deletion src/v/raft/tests/offset_translator_tests.cc
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,9 @@ struct base_fixture {
.invoke_on_all(
[](features::feature_table& f) { f.testing_activate_all(); })
.get();
_memory_sampling_service.start(std::ref(_test_logger)).get();
_memory_sampling_service
.start(std::ref(_test_logger), config::mock_binding<bool>(false))
.get();
_api
.start(
[this]() { return make_kv_cfg(); },
Expand Down
6 changes: 5 additions & 1 deletion src/v/redpanda/application.cc
Original file line number Diff line number Diff line change
Expand Up @@ -398,7 +398,11 @@ void application::initialize(
std::optional<YAML::Node> schema_reg_cfg,
std::optional<YAML::Node> schema_reg_client_cfg,
std::optional<scheduling_groups> groups) {
construct_service(_memory_sampling, std::ref(_log)).get();
construct_service(
_memory_sampling, std::ref(_log), ss::sharded_parameter([]() {
return config::shard_local_cfg().sampled_memory_profile.bind();
}))
.get();
_memory_sampling.invoke_on_all(&memory_sampling::start).get();

// Set up the abort_on_oom value based on the associated cluster config
Expand Down
41 changes: 32 additions & 9 deletions src/v/resource_mgmt/memory_sampling.cc
Original file line number Diff line number Diff line change
Expand Up @@ -124,27 +124,50 @@ ss::future<> memory_sampling::start_low_available_memory_logging() {
}
}

memory_sampling::memory_sampling(ss::logger& logger)
: _logger(logger)
, _first_log_limit_fraction(0.2)
, _second_log_limit_fraction(0.1) {}
memory_sampling::memory_sampling(
ss::logger& logger, config::binding<bool> enabled)
: memory_sampling(logger, std::move(enabled), 0.2, 0.1) {}

memory_sampling::memory_sampling(
ss::logger& logger,
config::binding<bool> enabled,
double first_log_limit_fraction,
double second_log_limit_fraction)
: _logger(logger)
, _enabled(std::move(enabled))
, _first_log_limit_fraction(first_log_limit_fraction)
, _second_log_limit_fraction(second_log_limit_fraction) {}

void memory_sampling::start() {
setup_additional_oom_diagnostics();
, _second_log_limit_fraction(second_log_limit_fraction) {
_enabled.watch([this]() { on_enabled_change(); });
}

void memory_sampling::on_enabled_change() {
// We chose a sampling rate of ~3MB. From testing this has a very low
// overhead of something like ~1%. We could still get away with something
// smaller like 1MB and have acceptable overhead (~3%) but 3MB should be a
// safer default for the initial rollout.
ss::memory::set_heap_profiling_sampling_rate(3000037);
const size_t sampling_rate = 3000037;

// Note no logging here as seastar already logs about this
if (_enabled()) {
if (ss::memory::get_heap_profiling_sample_rate() == sampling_rate) {
return;
}

ss::memory::set_heap_profiling_sampling_rate(sampling_rate);
} else {
if (ss::memory::get_heap_profiling_sample_rate() == 0) {
return;
}

ss::memory::set_heap_profiling_sampling_rate(0);
}
}

void memory_sampling::start() {
setup_additional_oom_diagnostics();

// start now if enabled
on_enabled_change();

ssx::spawn_with_gate(_low_watermark_gate, [this]() {
return start_low_available_memory_logging();
Expand Down
9 changes: 8 additions & 1 deletion src/v/resource_mgmt/memory_sampling.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@

#pragma once

#include "config/property.h"
#include "seastarx.h"

#include <seastar/core/condition-variable.hh>
Expand Down Expand Up @@ -85,12 +86,13 @@ class memory_sampling : public ss::peering_sharded_service<memory_sampling> {

/// Constructs the service. Logger will be used to log top stacks under high
/// memory pressure
explicit memory_sampling(ss::logger& logger);
explicit memory_sampling(ss::logger& logger, config::binding<bool> enabled);

/// Constructor as above but allows overriding high memory thresholds. Used
/// for testing.
explicit memory_sampling(
ss::logger& logger,
config::binding<bool> enabled,
double first_log_limit_fraction,
double second_log_limit_fraction);

Expand All @@ -108,8 +110,13 @@ class memory_sampling : public ss::peering_sharded_service<memory_sampling> {
static memory_sampling::serialized_memory_profile
get_sampled_memory_profile();

void on_enabled_change();

ss::logger& _logger;

/// Are we currently sampling memory
config::binding<bool> _enabled;

// When a memory reclaim from the seastar allocator happens the batch_cache
// notifies us via below condvar. If we see a new low watermark that's and
// we are below 20% then we log once and a second time the first time we saw
Expand Down
7 changes: 6 additions & 1 deletion src/v/resource_mgmt/tests/memory_sampling_reclaimer_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,12 @@ SEASTAR_THREAD_TEST_CASE(reclaim_notifies_memory_sampling) {

std::string_view needle("Top-N alloc");
const auto first_log_limit = 0.80;
memory_sampling_service.start(std::ref(test_logger), first_log_limit, 0.2)
memory_sampling_service
.start(
std::ref(test_logger),
config::mock_binding<bool>(true),
first_log_limit,
0.2)
.get();
memory_sampling_service.invoke_on_all(&memory_sampling::start).get();

Expand Down
6 changes: 5 additions & 1 deletion src/v/resource_mgmt/tests/memory_sampling_tests.cc
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,11 @@ SEASTAR_THREAD_TEST_CASE(test_low_watermark_logging) {
const auto first_log_limit = 0.90;
const auto second_log_limit = 0.80;

memory_sampling sampling(dummy_logger, first_log_limit, second_log_limit);
memory_sampling sampling(
dummy_logger,
config::mock_binding<bool>(true),
first_log_limit,
second_log_limit);
sampling.start();

std::string_view needle("Top-N alloc");
Expand Down
4 changes: 3 additions & 1 deletion src/v/storage/tests/batch_cache_reclaim_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,9 @@ FIXTURE_TEST(reclaim, fixture) {

seastar::logger test_logger("test");
ss::sharded<memory_sampling> memory_sampling_service;
memory_sampling_service.start(std::ref(test_logger)).get();
memory_sampling_service
.start(std::ref(test_logger), config::mock_binding<bool>(false))
.get();

storage::batch_cache cache(opts, memory_sampling_service);
storage::batch_cache_index index(cache);
Expand Down
12 changes: 9 additions & 3 deletions src/v/storage/tests/batch_cache_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,9 @@ class batch_cache_test_fixture {
batch_cache_test_fixture()
: test_logger("batch-cache-test")
, cache(opts, memory_sampling_service) {
memory_sampling_service.start(std::ref(test_logger)).get();
memory_sampling_service
.start(std::ref(test_logger), config::mock_binding<bool>(false))
.get();
}

auto& get_lru() { return cache._lru; };
Expand Down Expand Up @@ -145,7 +147,9 @@ SEASTAR_THREAD_TEST_CASE(touch) {

seastar::logger test_logger("test");
ss::sharded<memory_sampling> memory_sampling_service;
memory_sampling_service.start(std::ref(test_logger)).get();
memory_sampling_service
.start(std::ref(test_logger), config::mock_binding<bool>(false))
.get();
auto action = ss::defer(
[&memory_sampling_service] { memory_sampling_service.stop().get(); });

Expand All @@ -169,7 +173,9 @@ SEASTAR_THREAD_TEST_CASE(touch) {
// build the cache the same way
seastar::logger test_logger("test");
ss::sharded<memory_sampling> memory_sampling_service;
memory_sampling_service.start(std::ref(test_logger)).get();
memory_sampling_service
.start(std::ref(test_logger), config::mock_binding<bool>(false))
.get();
auto action = ss::defer(
[&memory_sampling_service] { memory_sampling_service.stop().get(); });

Expand Down
4 changes: 3 additions & 1 deletion src/v/storage/tests/log_manager_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,9 @@ SEASTAR_THREAD_TEST_CASE(test_can_load_logs) {
.get();

ss::sharded<memory_sampling> memory_sampling_service;
memory_sampling_service.start(std::ref(test_logger)).get();
memory_sampling_service
.start(std::ref(test_logger), config::mock_binding<bool>(false))
.get();

storage::api store(
[conf]() {
Expand Down
4 changes: 3 additions & 1 deletion src/v/storage/tests/storage_test_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -217,7 +217,9 @@ class storage_test_fixture {
.invoke_on_all(
[](features::feature_table& f) { f.testing_activate_all(); })
.get();
memory_sampling_service.start(std::ref(tlog)).get();
memory_sampling_service
.start(std::ref(tlog), config::mock_binding<bool>(false))
.get();

kvstore.start().get();
}
Expand Down
4 changes: 3 additions & 1 deletion src/v/storage/tests/utils/disk_log_builder.cc
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,9 @@ disk_log_builder::disk_log_builder(storage::log_config config)
[this]() { return _log_config; },
_feature_table,
_memory_sampling_service) {
_memory_sampling_service.start(std::ref(_test_logger)).get();
_memory_sampling_service
.start(std::ref(_test_logger), config::mock_binding<bool>(false))
.get();
}

// Batch generation
Expand Down
8 changes: 6 additions & 2 deletions src/v/test_utils/logs.h
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,9 @@ static inline ss::future<> persist_log_file(

seastar::logger test_logger("test");
ss::sharded<memory_sampling> memory_sampling_service;
memory_sampling_service.start(std::ref(test_logger)).get();
memory_sampling_service
.start(std::ref(test_logger), config::mock_binding<bool>(false))
.get();

ss::sharded<storage::api> storage;
storage
Expand Down Expand Up @@ -122,7 +124,9 @@ read_log_file(ss::sstring base_dir, model::ntp file_ntp) {

seastar::logger test_logger("test");
ss::sharded<memory_sampling> memory_sampling_service;
memory_sampling_service.start(std::ref(test_logger)).get();
memory_sampling_service
.start(std::ref(test_logger), config::mock_binding<bool>(false))
.get();

ss::sharded<storage::api> storage;
storage
Expand Down

0 comments on commit 6c0f1c0

Please sign in to comment.