Skip to content

Commit

Permalink
Merge pull request #24906 from vbotbuildovich/backport-pr-24837-v24.3…
Browse files Browse the repository at this point in the history
….x-319

[v24.3.x] Fix partitions local summary
  • Loading branch information
bashtanov authored Jan 27, 2025
2 parents 6e19abc + a612484 commit 0a67ba9
Show file tree
Hide file tree
Showing 3 changed files with 75 additions and 1 deletion.
2 changes: 1 addition & 1 deletion src/v/redpanda/admin/partition.cc
Original file line number Diff line number Diff line change
Expand Up @@ -724,7 +724,7 @@ void admin_server::register_partition_routes() {
if (it.second->get_leader_id() == std::nullopt) {
s.leaderless += 1;
}
if (it.second->get_under_replicated() == std::nullopt) {
if (it.second->get_under_replicated() > 0) {
s.under_replicated += 1;
}
}
Expand Down
4 changes: 4 additions & 0 deletions tests/rptest/services/admin.py
Original file line number Diff line number Diff line change
Expand Up @@ -1563,6 +1563,10 @@ def get_partition_state(self, namespace, topic, partition, node=None):
path = f"debug/partition/{namespace}/{topic}/{partition}"
return self._request("GET", path, node=node).json()

def get_partitions_local_summary(self, node: ClusterNode):
path = f"partitions/local_summary"
return self._request("GET", path, node=node).json()

def get_producers_state(self, namespace, topic, partition, node=None):
path = f"debug/producers/{namespace}/{topic}/{partition}"
return self._request("GET", path, node=node).json()
Expand Down
70 changes: 70 additions & 0 deletions tests/rptest/tests/partition_state_api_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,8 +10,13 @@
from rptest.services.admin import Admin
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.cluster import cluster
from rptest.clients.kafka_cli_tools import KafkaCliTools
from rptest.clients.types import TopicSpec

from collections import Counter
import time
import random


class PartitionStateAPItest(RedpandaTest):
def __init__(self, *args, **kwargs):
Expand Down Expand Up @@ -129,3 +134,68 @@ def test_controller_partition_state(self):
leader_state = leaders[0]["raft_state"]
assert "followers" in leader_state.keys() and len(
leader_state["followers"]) == 4

@cluster(num_nodes=5)
def test_local_summary(self):
admin = Admin(self.redpanda)
n_topics = 100
# allow for a couple of system partitions
tolerance = 2 * len(self.redpanda.nodes)

def produce():
kafka_tools = KafkaCliTools(self.redpanda)
for t in self.topics:
kafka_tools.produce(t.name, 1, 1, acks=1)

def sumsum_eventually(**expected):
self.logger.debug(f"{expected=}")

def check():
summaries = [
Counter(admin.get_partitions_local_summary(n))
for n in self.redpanda.started_nodes()
]
self.logger.debug(f"{summaries=}")
ss = sum(summaries, Counter())
for k, v in expected.items():
if ss[k] < v or ss[k] > v + tolerance:
return False
return True

self.redpanda.wait_until(check, 30, 2,
"Unexpected local partition summaries")

time.sleep(5)
assert check()

def stop_one():
node_to_stop = random.choice(self.redpanda.started_nodes())
self.logger.debug(
f"Stopping node {self.redpanda.idx(node_to_stop)}")
self.redpanda.stop_node(node_to_stop)

self.topics = [
TopicSpec(partition_count=1,
replication_factor=len(self.redpanda.nodes))
for _ in range(n_topics)
]
self._create_initial_topics()

# all 5 nodes live
produce()
sumsum_eventually(count=5 * n_topics, leaderless=0, under_replicated=0)

# 3 out of 5 nodes to remain live
stop_one()
stop_one()
produce()
sumsum_eventually(count=3 * n_topics,
leaderless=0,
under_replicated=n_topics)

# 2 out of 5 nodes to remain live
produce() # after stop we won't be able to produce into leaderless
stop_one()
sumsum_eventually(count=2 * n_topics,
leaderless=2 * n_topics,
under_replicated=0)

0 comments on commit 0a67ba9

Please sign in to comment.