Skip to content

Commit

Permalink
cluster: include a hash in the metrics reporter
Browse files Browse the repository at this point in the history
Signed-off-by: NyaliaLui <nyalia@redpanda.com>
  • Loading branch information
NyaliaLui committed Aug 9, 2023
1 parent a7f0f8b commit eccaf0e
Show file tree
Hide file tree
Showing 3 changed files with 26 additions and 1 deletion.
8 changes: 8 additions & 0 deletions src/v/cluster/metrics_reporter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,11 @@ metrics_reporter::build_metrics_snapshot() {
0, metrics_snapshot::max_size_for_rp_env);
}

auto license = _feature_table.local().get_license();
if (license.has_value()) {
snapshot.id_hash = license->checksum;
}

co_return snapshot;
}

Expand Down Expand Up @@ -523,6 +528,9 @@ void rjson_serialize(
w.Key("redpanda_environment");
w.String(snapshot.redpanda_environment);

w.Key("id_hash");
w.String(snapshot.id_hash);

w.EndObject();
}

Expand Down
1 change: 1 addition & 0 deletions src/v/cluster/metrics_reporter.h
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ class metrics_reporter {

static constexpr int64_t max_size_for_rp_env = 80;
ss::sstring redpanda_environment;
ss::sstring id_hash;
};
static constexpr ss::shard_id shard = 0;

Expand Down
18 changes: 17 additions & 1 deletion tests/rptest/tests/metrics_reporter_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@

from rptest.services.cluster import cluster
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST
from rptest.utils.rpenv import sample_license
from ducktape.utils.util import wait_until

from rptest.clients.types import TopicSpec
Expand Down Expand Up @@ -47,6 +48,18 @@ def _test_redpanda_metrics_reporting(self):
"""
Test that redpanda nodes send well formed messages to the metrics endpoint
"""

# Load and put a license at start. This is to check the SHA-256 checksum
admin = Admin(self.redpanda)
license = sample_license()
if license is None:
self.logger.warn(
"REDPANDA_SAMPLE_LICENSE env var not found, ignoring license checks."
)

assert admin.put_license(
license).status_code == 200, "PUT License failed"

total_topics = 5
total_partitions = 0
for _ in range(0, total_topics):
Expand Down Expand Up @@ -80,7 +93,6 @@ def _state_up_to_date():
def assert_fields_are_the_same(metadata, field):
assert all(m[field] == metadata[0][field] for m in metadata)

admin = Admin(self.redpanda)
features = admin.get_features()

# cluster uuid and create timestamp should stay the same across requests
Expand Down Expand Up @@ -123,6 +135,7 @@ def assert_fields_are_the_same(metadata, field):
wait_until(lambda: len(self.http.requests) > pre_restart_requests,
timeout_sec=20,
backoff_sec=1)
self.redpanda.logger.info("Checking metadata after restart")
assert_fields_are_the_same(metadata, 'cluster_uuid')
assert_fields_are_the_same(metadata, 'cluster_created_ts')

Expand All @@ -134,6 +147,9 @@ def assert_fields_are_the_same(metadata, field):
assert last["config"]["log_message_timestamp_type"] == "CreateTime"
assert last["redpanda_environment"] == "test"

last_post_restart = metadata.pop()
assert last_post_restart["id_hash"] == last["id_hash"]


class MultiNodeMetricsReporterTest(MetricsReporterTest):
"""
Expand Down

0 comments on commit eccaf0e

Please sign in to comment.