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


##########
clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java:
##########
@@ -335,6 +349,68 @@ public void testRackAwareAssignment(ClusterInstance 
clusterInstance) throws Exec
         }
     }
 
+    @ClusterTest(
+        brokers = 2,
+        types = {Type.KRAFT},
+        serverProperties = {
+            @ClusterConfigProperty(key = 
GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
+            @ClusterConfigProperty(key = 
GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, value = 
"3000")
+        }
+    )
+    public void 
testSingleCoordinatorOwnershipAfterPartitionReassignment(ClusterInstance 
clusterInstance)
+        throws InterruptedException, ExecutionException, TimeoutException, 
MalformedObjectNameException, ReflectionException, AttributeNotFoundException, 
InstanceNotFoundException, MBeanException {
+        try (var producer = clusterInstance.<byte[], byte[]>producer()) {
+            producer.send(new ProducerRecord<>("topic", 
"value".getBytes(StandardCharsets.UTF_8)));
+        }
+
+        try (var admin = clusterInstance.admin()) {
+            admin.createTopics(List.of(new 
NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, Map.of(0, List.of(0))))).all().get();
+        }
+
+        try (var consumer = 
clusterInstance.consumer(Map.of(ConsumerConfig.GROUP_ID_CONFIG, "test-group"));
+             var admin = clusterInstance.admin()) {
+            consumer.subscribe(List.of("topic"));
+            TestUtils.waitForCondition(() -> 
consumer.poll(Duration.ofMillis(100)).isEmpty(), "polling to join group");
+            // append records to coordinator
+            consumer.commitSync();
+
+            var numActivePartitions = (Long) 
ManagementFactory.getPlatformMBeanServer().getAttribute(
+                new 
ObjectName("kafka.server:type=group-coordinator-metrics,state=active"), 
"num-partitions");
+            // The num-partitions metric uses the same name for all brokers 
and is
+            // registered in the same JVM. Therefore, we infer the owning 
broker
+            // based on the number of active partitions.
+            // - activePartitions == 1 means metric belongs to broker 0 
(partition 0 assigned)
+            // - activePartitions == 0 means metric belongs to broker 1
+            AtomicBoolean isMetricBelongsToFirstBroker = new AtomicBoolean();
+            if (numActivePartitions == 1L) {
+                isMetricBelongsToFirstBroker.set(true);
+            } else if (numActivePartitions == 0L) {
+                isMetricBelongsToFirstBroker.set(false);
+            } else {
+                Assertions.fail("Unexpected number of active partitions: " + 
numActivePartitions);
+            }

Review Comment:
   I am sorry but I still don't like it. Is there another way to probe the 
coordinator? e.g. by accessing the metric registry directly with 
`clusterInstance.brokers().get(0).metrics().metric(...)` or access the group 
coordinator object?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to