Skip to content

Commit

Permalink
c/metrics: Added kafka_advertised_listeners to metrics report
Browse files Browse the repository at this point in the history
Signed-off-by: Michael Boquard <michael@redpanda.com>
  • Loading branch information
michael-redpanda committed Nov 6, 2024
1 parent c6df232 commit c69776c
Show file tree
Hide file tree
Showing 3 changed files with 19 additions and 28 deletions.
44 changes: 16 additions & 28 deletions src/v/cluster/metrics_reporter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
#include <seastar/core/coroutine.hh>
#include <seastar/core/lowres_clock.hh>
#include <seastar/core/shared_ptr.hh>
#include <seastar/net/dns.hh>
#include <seastar/net/tls.hh>
#include <seastar/util/defer.hh>

Expand Down Expand Up @@ -124,33 +125,10 @@ ss::sstring get_domainname() {
return domainname.data();
}

std::vector<ss::sstring> get_fqdn(std::string_view hostname) {
struct addrinfo hints = {};
struct addrinfo* info = nullptr;

memset(&hints, 0, sizeof(hints));
hints.ai_family = AF_UNSPEC;
hints.ai_socktype = SOCK_STREAM;
hints.ai_flags = AI_CANONNAME;

auto free_info = ss::defer([&info] {
if (info != nullptr) {
freeaddrinfo(info);
}
});

if (getaddrinfo(hostname.data(), nullptr, &hints, &info) != 0) {
return {};
}

std::vector<ss::sstring> rv;
for (auto p = info; p != nullptr; p = p->ai_next) {
if (p->ai_canonname != nullptr) {
rv.emplace_back(p->ai_canonname);
}
}

return rv;
ss::future<std::vector<ss::sstring>> get_fqdn(const ss::sstring& hostname) {
ss::net::dns_resolver resolver;
auto entries = co_await resolver.get_host_by_name(hostname);
co_return entries.names;
}
} // namespace details

Expand Down Expand Up @@ -269,6 +247,14 @@ metrics_reporter::build_metrics_snapshot() {
}

metrics.uptime_ms = report->local_state.uptime / 1ms;
auto& advertised_listeners
= nm->get().broker.kafka_advertised_listeners();
metrics.advertised_listeners.reserve(advertised_listeners.size());
std::transform(
advertised_listeners.begin(),
advertised_listeners.end(),
std::back_inserter(metrics.advertised_listeners),
[](const model::broker_endpoint& ep) { return ep.address; });
}
auto& topics = _topics.local().topics_map();
snapshot.topic_count = 0;
Expand Down Expand Up @@ -330,7 +316,7 @@ metrics_reporter::build_metrics_snapshot() {

snapshot.host_name = details::get_hostname();
snapshot.domain_name = details::get_domainname();
snapshot.fqdns = details::get_fqdn(snapshot.host_name);
snapshot.fqdns = co_await details::get_fqdn(snapshot.host_name);

co_return snapshot;
}
Expand Down Expand Up @@ -666,6 +652,8 @@ void rjson_serialize(
rjson_serialize(w, d);
}
w.EndArray();
w.Key("kafka_advertised_listeners");
rjson_serialize(w, nm.advertised_listeners);

w.EndObject();
}
Expand Down
2 changes: 2 additions & 0 deletions src/v/cluster/metrics_reporter.h
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
#include "model/metadata.h"
#include "security/fwd.h"
#include "utils/prefix_logger.h"
#include "utils/unresolved_address.h"

#include <seastar/core/abort_source.hh>
#include <seastar/core/gate.hh>
Expand Down Expand Up @@ -58,6 +59,7 @@ class metrics_reporter {
cluster_version logical_version{invalid_version};
std::vector<node_disk_space> disks;
uint64_t uptime_ms{0};
std::vector<net::unresolved_address> advertised_listeners;
};

struct metrics_snapshot {
Expand Down
1 change: 1 addition & 0 deletions tests/rptest/tests/metrics_reporter_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,7 @@ def assert_fields_are_the_same(metadata, field):
assert all('uptime_ms' in n for n in nodes_meta)
assert all('is_alive' in n for n in nodes_meta)
assert all('disks' in n for n in nodes_meta)
assert all('kafka_advertised_listeners' in n for n in nodes_meta)

# Check cluster UUID and creation time survive a restart
for n in self.redpanda.nodes:
Expand Down

0 comments on commit c69776c

Please sign in to comment.