Skip to content

Commit

Permalink
Merge pull request redpanda-data#20813 from mmaslankaprv/fix-20574
Browse files Browse the repository at this point in the history
tests: wait for the leadership change metric to be updated
  • Loading branch information
mmaslankaprv authored Jul 4, 2024
2 parents d6cbe67 + 2c009b1 commit 26b6202
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 15 deletions.
4 changes: 3 additions & 1 deletion tests/rptest/services/metrics_check.py
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,9 @@ def _capture(self, check_metrics):
samples[sample.name] = sample.value

for k, v in samples.items():
self.logger.info(f" Captured {k}={v}")
self.logger.info(
f" Captured {k}={v} from {self.node.account.hostname}(node_id = {self.redpanda.node_id(self.node)})"
)

if len(samples) == 0:
# Announce
Expand Down
42 changes: 28 additions & 14 deletions tests/rptest/tests/raft_availability_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -386,26 +386,40 @@ def test_leadership_transfer(self):
partition=0,
target_id=None,
leader_id=initial_leader_id)
new_leader_id, _ = self._wait_for_leader(
lambda l: l is not None and l != initial_leader_id)
hosts = [n.account.hostname for n in self.redpanda.nodes]
new_leader_id = admin.await_stable_leader(
topic=self.topic,
partition=0,
hosts=hosts,
check=lambda l: l is not None and l != initial_leader_id)
new_leader_node = self.redpanda.get_node_by_id(new_leader_id)
assert new_leader_node is not None
self.logger.info(
f"New leader is {new_leader_id} {new_leader_node.account.hostname}"
)

for [id, metric_check] in metric_checks.items():
# the metric should be updated only on the node that was elected as a leader
if id == new_leader_id:
metric_check.expect([
("vectorized_raft_leadership_changes_total",
lambda initial, current: current == initial + 1),
])
else:
metric_check.expect([
("vectorized_raft_leadership_changes_total",
lambda initial, current: current == initial),
])
def metrics_updated():
results = []
for [id, metric_check] in metric_checks.items():
# the metric should be updated only on the node that was elected as a leader
if id == new_leader_id:
results.append(
metric_check.evaluate([
("vectorized_raft_leadership_changes_total",
lambda initial, current: current == initial + 1),
]))
else:
results.append(
metric_check.evaluate([
("vectorized_raft_leadership_changes_total",
lambda initial, current: current == initial),
]))

return all(results)

wait_until(
metrics_updated, 30, 1,
"Leadership changes metric should be updated only on the leader")

@cluster(num_nodes=4)
@parametrize(acks=1)
Expand Down

0 comments on commit 26b6202

Please sign in to comment.