dajac commented on code in PR #12206:
URL: https://github.com/apache/kafka/pull/12206#discussion_r895790891


##########
clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.OptionalLong;
+
+/**
+ * This class is used to describe the state of the quorum received in 
DescribeQuorumResponse.
+ */
+public class QuorumInfo {
+    private final Integer leaderId;
+    private final List<ReplicaState> voters;
+    private final List<ReplicaState> observers;
+
+    QuorumInfo(Integer leaderId, List<ReplicaState> voters, List<ReplicaState> 
observers) {
+        this.leaderId = leaderId;
+        this.voters = voters;
+        this.observers = observers;
+    }
+
+    public Integer leaderId() {
+        return leaderId;
+    }
+
+    public List<ReplicaState> voters() {
+        return voters;
+    }
+
+    public List<ReplicaState> observers() {
+        return observers;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        QuorumInfo that = (QuorumInfo) o;
+        return leaderId.equals(that.leaderId)
+            && voters.equals(that.voters)
+            && observers.equals(that.observers);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(leaderId, voters, observers);
+    }
+
+    @Override
+    public String toString() {
+        return "QuorumInfo(" +
+            "leaderId=" + leaderId +
+            ", voters=" + voters.toString() +
+            ", observers=" + observers.toString() +
+            ')';
+    }
+
+    public static class ReplicaState {
+        private final int replicaId;
+        private final long logEndOffset;
+        private final OptionalLong lastFetchTimeMs;
+        private final OptionalLong lastCaughtUpTimeMs;
+
+        ReplicaState() {
+            this(0, 0, OptionalLong.empty(), OptionalLong.empty());
+        }
+
+        ReplicaState(
+                int replicaId,
+                long logEndOffset,
+                OptionalLong lastFetchTimeMs,
+                OptionalLong lastCaughtUpTimeMs

Review Comment:
   nit: We usually use 4 spaces indentation here.



##########
core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala:
##########
@@ -778,4 +778,44 @@ class KRaftClusterTest {
       cluster.close()
     }
   }
+
+  @Test
+  def testDescribeQuorumRequestToBrokers() : Unit = {
+    val cluster = new KafkaClusterTestKit.Builder(
+      new TestKitNodes.Builder().
+        setNumBrokerNodes(4).
+        setNumControllerNodes(3).build()).build()
+    try {
+      cluster.format
+      cluster.startup
+      for (i <- 0 to 3) {
+        TestUtils.waitUntilTrue(() => cluster.brokers.get(i).brokerState == 
BrokerState.RUNNING,
+          "Broker Never started up")
+      }
+      val props = cluster.clientProperties()
+      props.put(AdminClientConfig.CLIENT_ID_CONFIG, this.getClass.getName)
+      val admin = Admin.create(props)
+      try {
+        val quorumState = admin.describeMetadataQuorum(new 
DescribeMetadataQuorumOptions)
+        val quorumInfo = quorumState.quorumInfo.get()
+
+        assertEquals(0, quorumInfo.observers.size())
+        assertEquals(3, quorumInfo.voters.size())

Review Comment:
   nit: We could omit parenthesis after `size`. The same for the previous line.



##########
core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala:
##########
@@ -54,30 +54,48 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) {
 
   @ClusterTest
   def testDescribeQuorum(): Unit = {
-    val request = new DescribeQuorumRequest.Builder(
-      singletonRequest(KafkaRaftServer.MetadataPartition)
-    ).build()
-
-    val response = connectAndReceive[DescribeQuorumResponse](request)
-
-    assertEquals(Errors.NONE, Errors.forCode(response.data.errorCode))
-    assertEquals(1, response.data.topics.size)
-
-    val topicData = response.data.topics.get(0)
-    assertEquals(KafkaRaftServer.MetadataTopic, topicData.topicName)
-    assertEquals(1, topicData.partitions.size)
-
-    val partitionData = topicData.partitions.get(0)
-    assertEquals(KafkaRaftServer.MetadataPartition.partition, 
partitionData.partitionIndex)
-    assertEquals(Errors.NONE, Errors.forCode(partitionData.errorCode))
-    assertTrue(partitionData.leaderEpoch > 0)
-
-    val leaderId = partitionData.leaderId
-    assertTrue(leaderId > 0)
-
-    val leaderState = partitionData.currentVoters.asScala.find(_.replicaId == 
leaderId)
-      .getOrElse(throw new AssertionError("Failed to find leader among current 
voter states"))
-    assertTrue(leaderState.logEndOffset > 0)
+    for (version <- ApiKeys.DESCRIBE_QUORUM.allVersions.asScala) {
+      val request = new DescribeQuorumRequest.Builder(
+        singletonRequest(KafkaRaftServer.MetadataPartition)
+      ).build(version.toShort)
+      val response = connectAndReceive[DescribeQuorumResponse](request)
+
+      assertEquals(Errors.NONE, Errors.forCode(response.data.errorCode))
+      assertEquals(1, response.data.topics.size)
+
+      val topicData = response.data.topics.get(0)
+      assertEquals(KafkaRaftServer.MetadataTopic, topicData.topicName)
+      assertEquals(1, topicData.partitions.size)
+
+      val partitionData = topicData.partitions.get(0)
+      assertEquals(KafkaRaftServer.MetadataPartition.partition, 
partitionData.partitionIndex)
+      assertEquals(Errors.NONE, Errors.forCode(partitionData.errorCode))
+      assertTrue(partitionData.leaderEpoch > 0)
+
+      val leaderId = partitionData.leaderId
+      assertTrue(leaderId > 0)
+
+      val leaderState = partitionData.currentVoters.asScala.find(_.replicaId 
== leaderId)
+        .getOrElse(throw new AssertionError("Failed to find leader among 
current voter states"))
+      assertTrue(leaderState.logEndOffset > 0)
+
+      val voterData = partitionData.currentVoters().asScala
+      val observerData = partitionData.observers().asScala
+      assertEquals(1, voterData.size)
+      assertEquals(0, observerData.size)
+      voterData.foreach { state =>
+        assertTrue(0 < state.replicaId)
+        assertTrue(0 < state.logEndOffset())
+        assertEquals(-1, state.lastFetchTimestamp())
+        assertEquals(-1, state.lastCaughtUpTimestamp())

Review Comment:
   nit: We can also omit parenthesis for those..



##########
clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.OptionalLong;
+
+/**
+ * This class is used to describe the state of the quorum received in 
DescribeQuorumResponse.
+ */
+public class QuorumInfo {
+    private final Integer leaderId;
+    private final List<ReplicaState> voters;
+    private final List<ReplicaState> observers;
+
+    QuorumInfo(Integer leaderId, List<ReplicaState> voters, List<ReplicaState> 
observers) {
+        this.leaderId = leaderId;
+        this.voters = voters;
+        this.observers = observers;
+    }
+
+    public Integer leaderId() {
+        return leaderId;
+    }
+
+    public List<ReplicaState> voters() {
+        return voters;
+    }
+
+    public List<ReplicaState> observers() {
+        return observers;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        QuorumInfo that = (QuorumInfo) o;
+        return leaderId.equals(that.leaderId)
+            && voters.equals(that.voters)
+            && observers.equals(that.observers);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(leaderId, voters, observers);
+    }
+
+    @Override
+    public String toString() {
+        return "QuorumInfo(" +
+            "leaderId=" + leaderId +
+            ", voters=" + voters.toString() +
+            ", observers=" + observers.toString() +

Review Comment:
   nit: I suppose that we could remove the `toString` as they as implicit.



##########
core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala:
##########
@@ -725,4 +725,35 @@ class KRaftClusterTest {
       cluster.close()
     }
   }
+
+  @Test
+  def testDescribeQuorumRequestToBrokers() : Unit = {
+    val cluster = new KafkaClusterTestKit.Builder(
+      new TestKitNodes.Builder().
+        setNumBrokerNodes(4).
+        setNumControllerNodes(3).build()).build()
+    try {
+      cluster.format
+      cluster.startup
+      for (i <- 0 to 3) {
+        TestUtils.waitUntilTrue(() => cluster.brokers.get(i).brokerState == 
BrokerState.RUNNING,
+          "Broker Never started up")
+      }
+      val props = cluster.clientProperties()
+      props.put(AdminClientConfig.CLIENT_ID_CONFIG, this.getClass.getName)
+      val admin = Admin.create(props)
+      try {
+        val quorumState = admin.describeMetadataQuorum(new 
DescribeMetadataQuorumOptions)
+        val quorumInfo = quorumState.quorumInfo().get()
+
+        assertEquals(KafkaRaftServer.MetadataTopic, quorumInfo.topic())
+        assertEquals(3, quorumInfo.voters.size())
+        assertEquals(0, quorumInfo.observers.size())

Review Comment:
   @hachikuji Is it the expected behavior to have zero observers?



##########
core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala:
##########
@@ -54,30 +54,48 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) {
 
   @ClusterTest
   def testDescribeQuorum(): Unit = {
-    val request = new DescribeQuorumRequest.Builder(
-      singletonRequest(KafkaRaftServer.MetadataPartition)
-    ).build()
-
-    val response = connectAndReceive[DescribeQuorumResponse](request)
-
-    assertEquals(Errors.NONE, Errors.forCode(response.data.errorCode))
-    assertEquals(1, response.data.topics.size)
-
-    val topicData = response.data.topics.get(0)
-    assertEquals(KafkaRaftServer.MetadataTopic, topicData.topicName)
-    assertEquals(1, topicData.partitions.size)
-
-    val partitionData = topicData.partitions.get(0)
-    assertEquals(KafkaRaftServer.MetadataPartition.partition, 
partitionData.partitionIndex)
-    assertEquals(Errors.NONE, Errors.forCode(partitionData.errorCode))
-    assertTrue(partitionData.leaderEpoch > 0)
-
-    val leaderId = partitionData.leaderId
-    assertTrue(leaderId > 0)
-
-    val leaderState = partitionData.currentVoters.asScala.find(_.replicaId == 
leaderId)
-      .getOrElse(throw new AssertionError("Failed to find leader among current 
voter states"))
-    assertTrue(leaderState.logEndOffset > 0)
+    for (version <- ApiKeys.DESCRIBE_QUORUM.allVersions.asScala) {
+      val request = new DescribeQuorumRequest.Builder(
+        singletonRequest(KafkaRaftServer.MetadataPartition)
+      ).build(version.toShort)
+      val response = connectAndReceive[DescribeQuorumResponse](request)
+
+      assertEquals(Errors.NONE, Errors.forCode(response.data.errorCode))
+      assertEquals(1, response.data.topics.size)
+
+      val topicData = response.data.topics.get(0)
+      assertEquals(KafkaRaftServer.MetadataTopic, topicData.topicName)
+      assertEquals(1, topicData.partitions.size)
+
+      val partitionData = topicData.partitions.get(0)
+      assertEquals(KafkaRaftServer.MetadataPartition.partition, 
partitionData.partitionIndex)
+      assertEquals(Errors.NONE, Errors.forCode(partitionData.errorCode))
+      assertTrue(partitionData.leaderEpoch > 0)
+
+      val leaderId = partitionData.leaderId
+      assertTrue(leaderId > 0)
+
+      val leaderState = partitionData.currentVoters.asScala.find(_.replicaId 
== leaderId)
+        .getOrElse(throw new AssertionError("Failed to find leader among 
current voter states"))
+      assertTrue(leaderState.logEndOffset > 0)
+
+      val voterData = partitionData.currentVoters().asScala

Review Comment:
   nit: We could omit parenthesis after `currentVoters`. The same for the next 
line.



##########
core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala:
##########
@@ -54,30 +54,48 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) {
 
   @ClusterTest
   def testDescribeQuorum(): Unit = {
-    val request = new DescribeQuorumRequest.Builder(
-      singletonRequest(KafkaRaftServer.MetadataPartition)
-    ).build()
-
-    val response = connectAndReceive[DescribeQuorumResponse](request)
-
-    assertEquals(Errors.NONE, Errors.forCode(response.data.errorCode))
-    assertEquals(1, response.data.topics.size)
-
-    val topicData = response.data.topics.get(0)
-    assertEquals(KafkaRaftServer.MetadataTopic, topicData.topicName)
-    assertEquals(1, topicData.partitions.size)
-
-    val partitionData = topicData.partitions.get(0)
-    assertEquals(KafkaRaftServer.MetadataPartition.partition, 
partitionData.partitionIndex)
-    assertEquals(Errors.NONE, Errors.forCode(partitionData.errorCode))
-    assertTrue(partitionData.leaderEpoch > 0)
-
-    val leaderId = partitionData.leaderId
-    assertTrue(leaderId > 0)
-
-    val leaderState = partitionData.currentVoters.asScala.find(_.replicaId == 
leaderId)
-      .getOrElse(throw new AssertionError("Failed to find leader among current 
voter states"))
-    assertTrue(leaderState.logEndOffset > 0)
+    for (version <- ApiKeys.DESCRIBE_QUORUM.allVersions.asScala) {
+      val request = new DescribeQuorumRequest.Builder(
+        singletonRequest(KafkaRaftServer.MetadataPartition)
+      ).build(version.toShort)
+      val response = connectAndReceive[DescribeQuorumResponse](request)
+
+      assertEquals(Errors.NONE, Errors.forCode(response.data.errorCode))
+      assertEquals(1, response.data.topics.size)
+
+      val topicData = response.data.topics.get(0)
+      assertEquals(KafkaRaftServer.MetadataTopic, topicData.topicName)
+      assertEquals(1, topicData.partitions.size)
+
+      val partitionData = topicData.partitions.get(0)
+      assertEquals(KafkaRaftServer.MetadataPartition.partition, 
partitionData.partitionIndex)
+      assertEquals(Errors.NONE, Errors.forCode(partitionData.errorCode))
+      assertTrue(partitionData.leaderEpoch > 0)
+
+      val leaderId = partitionData.leaderId
+      assertTrue(leaderId > 0)
+
+      val leaderState = partitionData.currentVoters.asScala.find(_.replicaId 
== leaderId)
+        .getOrElse(throw new AssertionError("Failed to find leader among 
current voter states"))
+      assertTrue(leaderState.logEndOffset > 0)
+
+      val voterData = partitionData.currentVoters().asScala
+      val observerData = partitionData.observers().asScala
+      assertEquals(1, voterData.size)
+      assertEquals(0, observerData.size)
+      voterData.foreach { state =>
+        assertTrue(0 < state.replicaId)
+        assertTrue(0 < state.logEndOffset())
+        assertEquals(-1, state.lastFetchTimestamp())
+        assertEquals(-1, state.lastCaughtUpTimestamp())
+      }
+      observerData.foreach { state =>
+        assertTrue(0 < state.replicaId)
+        assertTrue(0 < state.logEndOffset())
+        assertEquals(-1, state.lastFetchTimestamp())
+        assertEquals(-1, state.lastCaughtUpTimestamp())
+      }

Review Comment:
   nit: Should we just remove this block if `assertEquals(0, 
observerData.size)`?



-- 
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

Reply via email to