Skip to content

Commit

Permalink
utils: refactor seastar_histogram_logform to allow for conversions fr…
Browse files Browse the repository at this point in the history
…om log_hist_internal to a public seastar hist

Co-authored-by: Ben Pope <[email protected]>
  • Loading branch information
ballard26 and BenPope committed Jul 28, 2023
1 parent 961178b commit 6b578bd
Show file tree
Hide file tree
Showing 5 changed files with 205 additions and 74 deletions.
2 changes: 1 addition & 1 deletion src/v/kafka/server/handlers/handler_probe.cc
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ void handler_probe::setup_metrics(
"latency_microseconds",
sm::description("Latency histogram of kafka requests"),
labels,
[this] { return _latency.seastar_histogram_logform(1); })
[this] { return _latency.internal_histogram_logform(); })
.aggregate(aggregate_labels),
});
}
Expand Down
1 change: 1 addition & 0 deletions src/v/utils/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ v_cc_library(
request_auth.cc
bottomless_token_bucket.cc
utf8.cc
log_hist.cc
DEPS
Seastar::seastar
Hdrhistogram::hdr_histogram
Expand Down
98 changes: 98 additions & 0 deletions src/v/utils/log_hist.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
* Copyright 2023 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/log_hist.h"

template<
typename duration_t,
int number_of_buckets,
uint64_t first_bucket_upper_bound>
template<typename cfg>
seastar::metrics::histogram
log_hist<duration_t, number_of_buckets, first_bucket_upper_bound>::
seastar_histogram_logform() const {
seastar::metrics::histogram hist;
hist.buckets.resize(cfg::bucket_count);
hist.sample_sum = static_cast<double>(_sample_sum)
/ static_cast<double>(cfg::scale);

const unsigned first_bucket_exp
= 64 - std::countl_zero(first_bucket_upper_bound - 1);
const unsigned cfg_first_bucket_exp
= 64 - std::countl_zero(cfg::first_bucket_bound - 1);

// Write bounds to seastar histogram
for (int i = 0; i < cfg::bucket_count; i++) {
auto& bucket = hist.buckets[i];
bucket.count = 0;

uint64_t unscaled_upper_bound = ((uint64_t)1
<< (cfg_first_bucket_exp + i))
- 1;
bucket.upper_bound = static_cast<double>(unscaled_upper_bound)
/ static_cast<double>(cfg::scale);
}

uint64_t cumulative_count = 0;
size_t current_hist_idx = 0;
double current_hist_upper_bound = hist.buckets[0].upper_bound;

// Write _counts to seastar histogram
for (size_t i = 0; i < _counts.size(); i++) {
uint64_t unscaled_upper_bound = ((uint64_t)1 << (first_bucket_exp + i))
- 1;
double scaled_upper_bound = static_cast<double>(unscaled_upper_bound)
/ static_cast<double>(cfg::scale);

cumulative_count += _counts[i];

while (scaled_upper_bound > current_hist_upper_bound
&& current_hist_idx != (hist.buckets.size() - 1)) {
current_hist_idx++;
current_hist_upper_bound
= hist.buckets[current_hist_idx].upper_bound;
}

hist.buckets[current_hist_idx].count = cumulative_count;
}

hist.sample_count = cumulative_count;
return hist;
}

template<
typename duration_t,
int number_of_buckets,
uint64_t first_bucket_upper_bound>
seastar::metrics::histogram
log_hist<duration_t, number_of_buckets, first_bucket_upper_bound>::
public_histogram_logform() const {
using public_hist_config = logform_config<1'000'000l, 256ul, 18>;

return seastar_histogram_logform<public_hist_config>();
}

template<
typename duration_t,
int number_of_buckets,
uint64_t first_bucket_upper_bound>
seastar::metrics::histogram
log_hist<duration_t, number_of_buckets, first_bucket_upper_bound>::
internal_histogram_logform() const {
using internal_hist_config = logform_config<1l, 8ul, 26>;

return seastar_histogram_logform<internal_hist_config>();
}

// Explicit instantiation for log_hist_public
template class log_hist<std::chrono::microseconds, 18, 256ul>;
// Explicit instantiation for log_hist_internal
template class log_hist<std::chrono::microseconds, 26, 8ul>;
69 changes: 36 additions & 33 deletions src/v/utils/log_hist.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,10 @@
#include <seastar/core/metrics_types.hh>
#include <seastar/core/shared_ptr.hh>

#include <boost/intrusive/list.hpp>

#include <array>
#include <bit>
#include <chrono>
#include <cstdint>
#include <vector>

/*
* A histogram implementation
Expand Down Expand Up @@ -112,7 +110,7 @@ class log_hist {

log_hist()
: _canary(seastar::make_lw_shared(true))
, _counts(number_of_buckets) {}
, _counts() {}
log_hist(const log_hist& o) = delete;
log_hist& operator=(const log_hist&) = delete;
log_hist(log_hist&& o) = delete;
Expand All @@ -135,40 +133,46 @@ class log_hist {
_counts[i]++;
}

void record(std::unique_ptr<measurement> m) {
record(m->compute_duration());
}

seastar::metrics::histogram seastar_histogram_logform(int64_t scale) const {
seastar::metrics::histogram hist;
hist.buckets.resize(_counts.size());
hist.sample_sum = static_cast<double>(_sample_sum)
/ static_cast<double>(scale);

uint64_t cumulative_count = 0;
for (uint64_t i = 0; i < _counts.size(); i++) {
auto& bucket = hist.buckets[i];

cumulative_count += _counts[i];
bucket.count = cumulative_count;
uint64_t unscaled_upper_bound = ((uint64_t)1
<< (first_bucket_exp + i))
- 1;
bucket.upper_bound = static_cast<double>(unscaled_upper_bound)
/ static_cast<double>(scale);
}
template<int64_t _scale, uint64_t _first_bucket_bound, int _bucket_count>
struct logform_config {
static constexpr auto bound_is_pow_2 = _first_bucket_bound >= 1
&& (_first_bucket_bound
& (_first_bucket_bound - 1))
== 0;
static_assert(
bound_is_pow_2, "_first_bucket_bound must be a power of 2");

static constexpr auto scale = _scale;
static constexpr auto first_bucket_bound = _first_bucket_bound;
static constexpr auto bucket_count = _bucket_count;
};

hist.sample_count = cumulative_count;
return hist;
}
template<typename cfg>
seastar::metrics::histogram seastar_histogram_logform() const;
/*
* Generates a Prometheus histogram with 18 buckets. The first bucket has an
* upper bound of 256 - 1 and subsequent buckets have an upper bound of 2
* times the upper bound of the previous bucket.
*
* This is the histogram type used in the `/public_metrics` endpoint
*/
seastar::metrics::histogram public_histogram_logform() const;
/*
* Generates a Prometheus histogram with 26 buckets. The first bucket has an
* upper bound of 8 - 1 and subsequent buckets have an upper bound of 2
* times the upper bound of the previous bucket.
*
* This is the histogram type used in the `/metrics` endpoint
*/
seastar::metrics::histogram internal_histogram_logform() const;

private:
friend measurement;

// Used to inform measurements whether `log_hist` has been destroyed
measurement_canary_t _canary;

std::vector<uint64_t> _counts;
std::array<uint64_t, number_of_buckets> _counts;
uint64_t _sample_sum{0};
};

Expand All @@ -179,13 +183,12 @@ class log_hist {
* will produce the same seastar histogram as
* `ssx::metrics::report_default_histogram(hdr_hist)`.
*/
using log_hist_public = log_hist<std::chrono::microseconds, 18, 256>;
static constexpr int64_t log_hist_public_scale = 1'000'000;
using log_hist_public = log_hist<std::chrono::microseconds, 18, 256ul>;

/*
* This histogram produces results that are similar, but not indentical to the
* internal metric's `hdr_hist`. Some of the first buckets will have the
* following bounds; [log_hist_internal upper bounds, internal hdr_hist upper
* bounds] [8, 10], [16, 20], [32, 41], [64, 83], [128, 167], [256, 335]
*/
using log_hist_internal = log_hist<std::chrono::microseconds, 26, 8>;
using log_hist_internal = log_hist<std::chrono::microseconds, 26, 8ul>;
109 changes: 69 additions & 40 deletions src/v/utils/tests/seastar_histogram_test.cc
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
#include "ssx/metrics.h"
#include "utils/hdr_hist.h"
#include "utils/log_hist.h"

Expand Down Expand Up @@ -34,51 +35,28 @@ bool approximately_equal(double a, double b) {
return std::abs(a - b) <= precision_error;
}

struct hist_config {
int64_t scale;
bool use_approximately_equal;
};
template<typename l_hist>
void validate_public_histograms_equal(const hdr_hist& a, const l_hist& b) {
const auto logform_a = ssx::metrics::report_default_histogram(a);
const auto logform_b = b.public_histogram_logform();

constexpr std::array hist_configs = {
hist_config{log_hist_public_scale, true}, hist_config{1, false}};
BOOST_CHECK_EQUAL(logform_a.sample_count, logform_b.sample_count);
BOOST_CHECK(
approximately_equal(logform_a.sample_sum, logform_b.sample_sum));

template<typename l_hist>
void validate_histograms_equal(const hdr_hist& a, const l_hist& b) {
for (auto cfg : hist_configs) {
const auto logform_a = a.seastar_histogram_logform(
18, 250, 2.0, cfg.scale);
const auto logform_b = b.seastar_histogram_logform(cfg.scale);

BOOST_CHECK_EQUAL(logform_a.sample_count, logform_b.sample_count);
if (cfg.use_approximately_equal) {
BOOST_CHECK(
approximately_equal(logform_a.sample_sum, logform_b.sample_sum));
} else {
BOOST_CHECK_EQUAL(logform_a.sample_sum, logform_b.sample_sum);
}

for (size_t idx = 0; idx < logform_a.buckets.size(); ++idx) {
if (cfg.use_approximately_equal) {
BOOST_CHECK(approximately_equal(
logform_a.buckets[idx].upper_bound,
logform_b.buckets[idx].upper_bound));
} else {
BOOST_CHECK_EQUAL(
logform_a.buckets[idx].upper_bound,
logform_b.buckets[idx].upper_bound);
}
BOOST_CHECK_EQUAL(
logform_a.buckets[idx].count, logform_b.buckets[idx].count);
}
for (size_t idx = 0; idx < logform_a.buckets.size(); ++idx) {
BOOST_CHECK(approximately_equal(
logform_a.buckets[idx].upper_bound,
logform_b.buckets[idx].upper_bound));
BOOST_CHECK_EQUAL(
logform_a.buckets[idx].count, logform_b.buckets[idx].count);
}
}
} // namespace

// ensures both the log_hist_public and the public hdr_hist return identical
// seastar histograms for values recorded around bucket bounds.
SEASTAR_THREAD_TEST_CASE(test_public_log_hist_and_hdr_hist_equal_bounds) {
using namespace std::chrono_literals;

hdr_hist a;
log_hist_public b;

Expand All @@ -94,14 +72,12 @@ SEASTAR_THREAD_TEST_CASE(test_public_log_hist_and_hdr_hist_equal_bounds) {
b.record(upper_bound + 1);
}

validate_histograms_equal(a, b);
validate_public_histograms_equal(a, b);
}

// ensures both the log_hist_public and the public hdr_hist return identical
// seastar histograms for randomly selected values.
SEASTAR_THREAD_TEST_CASE(test_public_log_hist_and_hdr_hist_equal_rand) {
using namespace std::chrono_literals;

hdr_hist a;
log_hist_public b;

Expand All @@ -115,5 +91,58 @@ SEASTAR_THREAD_TEST_CASE(test_public_log_hist_and_hdr_hist_equal_rand) {
b.record(sample);
}

validate_histograms_equal(a, b);
validate_public_histograms_equal(a, b);
}

// Ensures that an internal histogram is properly converted to a public metrics
// histogram.
SEASTAR_THREAD_TEST_CASE(test_internal_hist_to_public_hist_bounds) {
hdr_hist a;
log_hist_internal b;

a.record(1);
b.record(1);

for (unsigned i = 0; i < 17; i++) {
auto upper_bound
= (((unsigned)1 << (log_hist_internal::first_bucket_exp + i)) - 1);
a.record(upper_bound);
a.record(upper_bound + 1);
b.record(upper_bound);
b.record(upper_bound + 1);
}

validate_public_histograms_equal(a, b);
}

// Ensures that generating a internal seastar histogram from log_hist_public
// results in the additional buckets for the extended lower bounds having counts
// of zero.
SEASTAR_THREAD_TEST_CASE(test_public_hist_to_internal_hist) {
log_hist_public a;
log_hist_internal b;

a.record(1);
b.record(1);

for (unsigned i = 0; i < 17; i++) {
auto upper_bound
= (((unsigned)1 << (log_hist_internal::first_bucket_exp + i)) - 1);
a.record(upper_bound);
a.record(upper_bound + 1);
b.record(upper_bound);
b.record(upper_bound + 1);
}

auto pub_to_int_hist = a.internal_histogram_logform();
auto int_to_int_hist = b.internal_histogram_logform();

const auto public_ub_exp = 8;
const auto internal_ub_exp = 3;

// The buckets in the extended lower bounds should be empty
for (int i = 0; i < public_ub_exp - internal_ub_exp; i++) {
BOOST_CHECK_EQUAL(pub_to_int_hist.buckets[i].count, 0);
BOOST_CHECK_NE(int_to_int_hist.buckets[i].count, 0);
}
}

0 comments on commit 6b578bd

Please sign in to comment.