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


##########
clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerIntegrationTest.java:
##########
@@ -335,6 +343,50 @@ public void testRackAwareAssignment(ClusterInstance 
clusterInstance) throws Exec
         }
     }
 
+    @SuppressWarnings("unchecked")
+    @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 {
+        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();
+
+            // unload the coordinator by changing leader (0 -> 1)
+            admin.alterPartitionReassignments(Map.of(new 
TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0),
+                Optional.of(new 
NewPartitionReassignment(List.of(1))))).all().get();
+        }

Review Comment:
   Yeah, it could be more robust. I’ve made some changes to the test after 
addressing this comment: 
https://github.com/apache/kafka/pull/21396#discussion_r2775606833. Please let 
me know if you have any concerns about the latest changes. Thanks!



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