[GitHub] [kafka] dengziming commented on a diff in pull request #12206: KAFKA-13888: Addition of Information in DescribeQuorumResponse about Voter Lag
dengziming commented on code in PR #12206: URL: https://github.com/apache/kafka/pull/12206#discussion_r889424223 ## clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java: ## @@ -0,0 +1,149 @@ +/* + * 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 is used to describe per-partition state in the DescribeQuorumResponse. + */ +public class QuorumInfo { +private final String topic; Review Comment: I think It's better to add partition here because we are making way for multi-raft. -- 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
[GitHub] [kafka] dengziming commented on a diff in pull request #12206: KAFKA-13888: Addition of Information in DescribeQuorumResponse about Voter Lag
dengziming commented on code in PR #12206: URL: https://github.com/apache/kafka/pull/12206#discussion_r889423970 ## clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java: ## @@ -4833,6 +4879,92 @@ public void testDescribeFeaturesFailure() { } } +@Test +public void testDescribeMetadataQuorumSuccess() throws Exception { +try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(ApiKeys.DESCRIBE_QUORUM.id, Review Comment: We can't use `NodeApiVersions.create()` here due to a bug here, which I'm trying to fix here: https://github.com/apache/kafka/pull/11784 -- 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
[GitHub] [kafka] dengziming commented on a diff in pull request #12206: KAFKA-13888: Addition of Information in DescribeQuorumResponse about Voter Lag
dengziming commented on code in PR #12206: URL: https://github.com/apache/kafka/pull/12206#discussion_r886641824 ## 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: I find the cause here, We set nodeId=-1 if it's a broker so observers.size==0 https://github.com/apache/kafka/blob/4c9eeef5b2dff9a4f0977fbc5ac7eaaf930d0d0e/core/src/main/scala/kafka/raft/RaftManager.scala#L185-L189 I changed it to `val nodeId = OptionalInt.of(config.nodeId)`, then observers.size==4 -- 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
[GitHub] [kafka] dengziming commented on a diff in pull request #12206: KAFKA-13888: Addition of Information in DescribeQuorumResponse about Voter Lag
dengziming commented on code in PR #12206: URL: https://github.com/apache/kafka/pull/12206#discussion_r885406422 ## 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: I wonder why there isn't an observer, we should have 4 observers since we have 4 brokers, this may also not be related to this PR, I will spend some time investigating. -- 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
[GitHub] [kafka] dengziming commented on a diff in pull request #12206: KAFKA-13888: Addition of Information in DescribeQuorumResponse about Voter Lag
dengziming commented on code in PR #12206: URL: https://github.com/apache/kafka/pull/12206#discussion_r882483263 ## core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala: ## @@ -725,4 +725,34 @@ class KRaftClusterTest { cluster.close() } } + + @Test + def testDescribeQuorumRequestToBrokers() = { Review Comment: nit: add : Unit after method ## core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala: ## @@ -725,4 +725,34 @@ class KRaftClusterTest { cluster.close() } } + + @Test + def testDescribeQuorumRequestToBrokers() = { +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()) Review Comment: we can also check observers here. -- 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
[GitHub] [kafka] dengziming commented on a diff in pull request #12206: KAFKA-13888: Addition of Information in DescribeQuorumResponse about Voter Lag
dengziming commented on code in PR #12206: URL: https://github.com/apache/kafka/pull/12206#discussion_r882261419 ## core/src/test/scala/unit/kafka/server/DescribeQuorumIntegrationTest.scala: ## @@ -26,16 +25,17 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.DescribeQuorumRequest.singletonRequest import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, ApiVersionsRequest, ApiVersionsResponse, DescribeQuorumRequest, DescribeQuorumResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Tag +import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag +@Timeout(120) @ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(clusterType = Type.KRAFT) @Tag("integration") -class DescribeQuorumRequestTest(cluster: ClusterInstance) { +class DescribeQuorumTest(cluster: ClusterInstance) { Review Comment: The class name and file name are inconsistent. ## clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java: ## @@ -4321,6 +4327,58 @@ void handleFailure(Throwable throwable) { return new UpdateFeaturesResult(new HashMap<>(updateFutures)); } +@Override +public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuorumOptions options) { +NodeProvider provider = new LeastLoadedNodeProvider(); + +final KafkaFutureImpl future = new KafkaFutureImpl<>(); +final long now = time.milliseconds(); +final Call call = new Call( +"describeMetadataQuorum", calcDeadlineMs(now, options.timeoutMs()), provider) { + +private QuorumInfo createQuorumResult(final DescribeQuorumResponse response) { +Integer partition = 0; +String topicName = response.getTopicNameByIndex(partition); Review Comment: `response.getTopicNameByIndex(partition)` is a little confusing here, how can we get a topic by partition, I think we should rename partition to topicIndex or use `response.getTopicNameByIndex(0)` directly. ## core/src/test/scala/unit/kafka/server/DescribeQuorumTest.scala: ## @@ -0,0 +1,106 @@ +/* + * 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 kafka.server + +import java.io.IOException +import kafka.test.ClusterInstance +import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.utils.NotNothing +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.requests.DescribeQuorumRequest.singletonRequest +import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, ApiVersionsRequest, ApiVersionsResponse, DescribeQuorumRequest, DescribeQuorumResponse} +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.{Tag, Timeout} +import org.junit.jupiter.api.extension.ExtendWith +import org.slf4j.LoggerFactory + +import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag + +@Timeout(120) +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +@ClusterTestDefaults(clusterType = Type.KRAFT) +@Tag("integration") +class DescribeQuorumIntegrationTest(cluster: ClusterInstance) { Review Comment: Ditto, The class name and file name are inconsistent. ## clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java: ## @@ -4846,6 +4878,31 @@ public void testDescribeFeaturesFailure() { } } +@Test +public void testDescribeMetadataQuorumSuccess() throws Exception { +try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create((short) 55, (short) 0, (short) 1)); Review Comment: ApiKeys.DESCRIBE_QUORUM.id, DESCRIBE_QUORUM.highestSupportVersion, DESCRIBE_QUORUM.lowestSupportVersion may be better here. ## core/src/test/scala/unit/kafka/server/DescribeQuorumTest.scala: ## @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under