apalan60 commented on code in PR #20420:
URL: https://github.com/apache/kafka/pull/20420#discussion_r2311828266


##########
clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/DescribeProducersWithBrokerIdTest.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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 org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
+import org.apache.kafka.common.test.ClusterInstance;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestDefaults;
+import org.apache.kafka.test.TestUtils;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.util.List;
+
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.server.config.ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+
+@ClusterTestDefaults(
+    brokers = 4,
+    serverProperties = {
+        @ClusterConfigProperty(key = AUTO_CREATE_TOPICS_ENABLE_CONFIG, value = 
"false"),
+        @ClusterConfigProperty(key = OFFSETS_TOPIC_PARTITIONS_CONFIG, value = 
"1"),
+        @ClusterConfigProperty(key = GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 
value = "0")
+    }
+)
+class DescribeProducersWithBrokerIdTest {
+    private static final String TOPIC_NAME = "test-topic";
+    private static final int NUM_PARTITIONS = 1;
+    private static final short REPLICATION_FACTOR = 3;
+
+    private final ClusterInstance clusterInstance;
+    private final TopicPartition topicPartition;
+
+    public DescribeProducersWithBrokerIdTest(ClusterInstance clusterInstance) {
+        this.clusterInstance = clusterInstance;
+        this.topicPartition = new TopicPartition(TOPIC_NAME, 0);
+    }
+
+    private static void sendTestRecords(Producer<byte[], byte[]> producer) {
+        producer.send(new ProducerRecord<>(TOPIC_NAME, 0, "key-0".getBytes(), 
"value-0".getBytes()));
+        producer.flush();
+    }
+    
+    @BeforeEach
+    void setUp() throws InterruptedException {
+        clusterInstance.createTopic(TOPIC_NAME, NUM_PARTITIONS, 
REPLICATION_FACTOR);
+    }
+
+    @ClusterTest
+    void testDescribeProducersDefaultRoutesToLeader() throws Exception {
+        try (Producer<byte[], byte[]> producer = clusterInstance.producer();
+             var admin = clusterInstance.admin()) {
+            sendTestRecords(producer);
+
+            var stateWithExplicitLeader = 
admin.describeProducers(List.of(topicPartition), 
+                    new 
DescribeProducersOptions().brokerId(clusterInstance.getLeaderBrokerId(topicPartition)))
+                    .partitionResult(topicPartition).get();
+            
+            var stateWithDefaultRouting = 
admin.describeProducers(List.of(topicPartition))
+                    .partitionResult(topicPartition).get();
+            
+            assertNotNull(stateWithDefaultRouting);
+            assertFalse(stateWithDefaultRouting.activeProducers().isEmpty());
+            assertEquals(stateWithExplicitLeader.activeProducers(), 
stateWithDefaultRouting.activeProducers());
+        }
+    }
+
+    @ClusterTest
+    void testDescribeProducersFromFollower() throws Exception {
+        try (Producer<byte[], byte[]> producer = clusterInstance.producer();
+             var admin = clusterInstance.admin()) {
+            sendTestRecords(producer);
+
+            var topicDescription = 
admin.describeTopics(List.of(topicPartition.topic())).allTopicNames().get().get(topicPartition.topic());
+            var replicaBrokerIds = 
topicDescription.partitions().get(topicPartition.partition()).replicas().stream()
+                    .map(Node::id)
+                    .toList();
+
+            var leaderBrokerId = 
clusterInstance.getLeaderBrokerId(topicPartition);
+            var followerBrokerId = replicaBrokerIds.stream()
+                    .filter(id -> id != leaderBrokerId)
+                    .findFirst()
+                    .orElseThrow(() -> new IllegalStateException("No follower 
found for partition " + topicPartition));
+            
+            var followerState = 
admin.describeProducers(List.of(topicPartition), 
+                    new DescribeProducersOptions().brokerId(followerBrokerId))
+                    .partitionResult(topicPartition).get();
+            var leaderState = admin.describeProducers(List.of(topicPartition))
+                    .partitionResult(topicPartition).get();
+
+            assertNotNull(followerState);
+            assertFalse(followerState.activeProducers().isEmpty());
+            assertEquals(leaderState.activeProducers(), 
followerState.activeProducers());
+        }
+    }
+
+    @ClusterTest
+    void testDescribeProducersWithInvalidBrokerId() throws Exception {
+        try (Producer<byte[], byte[]> producer = clusterInstance.producer();
+             var admin = clusterInstance.admin()) {
+            sendTestRecords(producer);
+
+            var topicDescription = 
admin.describeTopics(List.of(topicPartition.topic())).allTopicNames().get().get(topicPartition.topic());
+            var replicaBrokerIds = 
topicDescription.partitions().get(topicPartition.partition()).replicas().stream()
+                    .map(Node::id)
+                    .toList();
+
+            var nonReplicaBrokerId = clusterInstance.brokerIds().stream()
+                    .filter(id -> !replicaBrokerIds.contains(id))

Review Comment:
   Updated.



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