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


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

Review Comment:
   
   Actually, the test does fail without the fix. The unloading logic is 
interrupted if `failCurrentBatch` throws an exception, so `onUnloaded` is never 
called
   
   ```java
   failCurrentBatch(Errors.NOT_COORDINATOR.exception()); // <--- Throws 
exception here, so the following logic is skipped
   if (coordinator != null) {
       try {
           coordinator.onUnloaded();
       } catch (Throwable ex) {
           log.error("Failed to unload coordinator for {} due to {}.", tp, 
ex.getMessage(), ex);
       }
   }
   coordinator = null; // <--- The coordinator is never cleared
   ```



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