showuon commented on code in PR #16106: URL: https://github.com/apache/kafka/pull/16106#discussion_r1629419251
########## clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java: ########## @@ -73,12 +83,13 @@ public boolean equals(Object o) { && leaderEpoch == that.leaderEpoch && highWatermark == that.highWatermark && Objects.equals(voters, that.voters) - && Objects.equals(observers, that.observers); + && Objects.equals(observers, that.observers) + && Objects.equals(nodes, that.nodes); Review Comment: Will the equals work as expected if we don't override equals method? ########## core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala: ########## @@ -69,6 +70,7 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) { val partitionData = topicData.partitions.get(0) assertEquals(KafkaRaftServer.MetadataPartition.partition, partitionData.partitionIndex) assertEquals(Errors.NONE, Errors.forCode(partitionData.errorCode)) + assertEquals("", partitionData.errorMessage()) Review Comment: ditto ########## clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java: ########## @@ -89,26 +100,30 @@ public String toString() { ", highWatermark=" + highWatermark + ", voters=" + voters + ", observers=" + observers + + ", nodes=" + nodes + Review Comment: Node object cannot be correct printed I think. No? ########## core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala: ########## @@ -60,6 +60,7 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) { val response = connectAndReceive[DescribeQuorumResponse](request) assertEquals(Errors.NONE, Errors.forCode(response.data.errorCode)) + assertEquals("", response.data.errorMessage) Review Comment: In `DescribeQuorumResponse.json`, we said `null if no error` for the error message, but here it gets empty string. Where do we do that conversion? ########## clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java: ########## @@ -29,19 +32,22 @@ public class QuorumInfo { private final long highWatermark; private final List<ReplicaState> voters; private final List<ReplicaState> observers; + private final Map<Integer, Node> nodes; Review Comment: Could you explain why we need a map here , not a Set<Node> ? It looks like we already store nodeId in Node object, and we don't need the nodeId as key, right? ########## raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java: ########## @@ -44,25 +48,26 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class LeaderStateTest { - private final int localId = 0; - private final int epoch = 5; + private final Entry<Integer, VoterSet.VoterNode> localId = entry(0); + private final Entry<Integer, VoterSet.VoterNode> epoch = entry(5); Review Comment: 1. It's not common to create an entry object 2. I don't think directryId has any effect in most of these test cases, so could we use a dummy one to represent it? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org