Skip to content

Commit

Permalink
storage: define storage::log_manager_probe
Browse files Browse the repository at this point in the history
  • Loading branch information
nvartolomei committed Nov 6, 2024
1 parent eba992f commit 1cd18e8
Show file tree
Hide file tree
Showing 6 changed files with 84 additions and 1 deletion.
15 changes: 15 additions & 0 deletions src/v/storage/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -356,6 +356,7 @@ redpanda_cc_library(
"version.h",
],
implementation_deps = [
":log_manager_probe",
":logger",
"//src/v/syschecks",
],
Expand Down Expand Up @@ -441,3 +442,17 @@ redpanda_cc_library(
"@seastar",
],
)

redpanda_cc_library(
name = "log_manager_probe",
srcs = [
"log_manager_probe.cc",
],
hdrs = [
"log_manager_probe.h",
],
implementation_deps = [
"//src/v/config",
],
include_prefix = "storage",
)
1 change: 1 addition & 0 deletions src/v/storage/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ v_cc_library(
segment_reader.cc
segment_deduplication_utils.cc
log_manager.cc
log_manager_probe.cc
disk_log_impl.cc
disk_log_appender.cc
parser.cc
Expand Down
9 changes: 8 additions & 1 deletion src/v/storage/log_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
#include "storage/key_offset_map.h"
#include "storage/kvstore.h"
#include "storage/log.h"
#include "storage/log_manager_probe.h"
#include "storage/logger.h"
#include "storage/segment.h"
#include "storage/segment_appender.h"
Expand Down Expand Up @@ -143,14 +144,17 @@ log_manager::log_manager(
, _feature_table(feature_table)
, _jitter(_config.compaction_interval())
, _trigger_gc_jitter(0s, 5s)
, _batch_cache(_config.reclaim_opts) {
, _batch_cache(_config.reclaim_opts)
, _probe(std::make_unique<log_manager_probe>()) {
_config.compaction_interval.watch([this]() {
_jitter = simple_time_jitter<ss::lowres_clock>{
_config.compaction_interval()};
_housekeeping_sem.signal();
});
}

log_manager::~log_manager() = default;

ss::future<> log_manager::clean_close(ss::shared_ptr<storage::log> log) {
auto clean_segment = co_await log->close();

Expand All @@ -171,6 +175,7 @@ ss::future<> log_manager::clean_close(ss::shared_ptr<storage::log> log) {
}

ss::future<> log_manager::start() {
_probe->setup_metrics();
if (unlikely(config::shard_local_cfg()
.log_disable_housekeeping_for_tests.value())) {
co_return;
Expand All @@ -195,6 +200,8 @@ ss::future<> log_manager::stop() {
co_await _compaction_hash_key_map->initialize(0);
_compaction_hash_key_map.reset();
}

_probe->clear_metrics();
}

/**
Expand Down
7 changes: 7 additions & 0 deletions src/v/storage/log_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,8 @@

namespace storage {

class log_manager_probe;

namespace testing_details {
class log_manager_accessor;
};
Expand Down Expand Up @@ -170,6 +172,7 @@ class log_manager {
kvstore& kvstore,
storage_resources&,
ss::sharded<features::feature_table>&) noexcept;
~log_manager();

ss::future<ss::shared_ptr<log>> manage(
ntp_config,
Expand Down Expand Up @@ -293,6 +296,10 @@ class log_manager {
// Hash key-map to use across multiple compactions to reuse reserved memory
// rather than reallocating repeatedly.
std::unique_ptr<hash_key_offset_map> _compaction_hash_key_map;

// Metrics.
std::unique_ptr<log_manager_probe> _probe;

ss::gate _gate;
ss::abort_source _abort_source;

Expand Down
24 changes: 24 additions & 0 deletions src/v/storage/log_manager_probe.cc
Original file line number Diff line number Diff line change
@@ -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 "storage/log_manager_probe.h"

#include "config/configuration.h"

namespace storage {

void log_manager_probe::setup_metrics() {
if (config::shard_local_cfg().disable_metrics()) {
return;
}
}

void log_manager_probe::clear_metrics() { _metrics.clear(); }

} // namespace storage
29 changes: 29 additions & 0 deletions src/v/storage/log_manager_probe.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
// 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

namespace storage {

/// Log manager per-shard storage probe.
class log_manager_probe {
public:
log_manager_probe() = default;
log_manager_probe(const log_manager_probe&) = delete;
log_manager_probe& operator=(const log_manager_probe&) = delete;
log_manager_probe(log_manager_probe&&) = delete;
log_manager_probe& operator=(log_manager_probe&&) = delete;
~log_manager_probe() = default;

public:
void setup_metrics();
void clear_metrics();
};

}; // namespace storage

0 comments on commit 1cd18e8

Please sign in to comment.