showuon commented on code in PR #15521:
URL: https://github.com/apache/kafka/pull/15521#discussion_r1544325278


##########
server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java:
##########
@@ -310,6 +301,62 @@ void testOnCompletion() throws Exception {
         }
     }
 
+    private static ClientResponse 
buildSuccessfulResponse(AssignReplicasToDirsRequestData request) {
+        Map<Uuid, Map<TopicIdPartition, Errors>> errors = new HashMap<>();
+        for (AssignReplicasToDirsRequestData.DirectoryData directory : 
request.directories()) {
+            for (AssignReplicasToDirsRequestData.TopicData topic : 
directory.topics()) {
+                for (AssignReplicasToDirsRequestData.PartitionData partition : 
topic.partitions()) {
+                    TopicIdPartition topicIdPartition = new 
TopicIdPartition(topic.topicId(), partition.partitionIndex());
+                    errors.computeIfAbsent(directory.id(), d -> new 
HashMap<>()).put(topicIdPartition, Errors.NONE);
+                }
+            }
+        }
+        AssignReplicasToDirsResponseData responseData = 
AssignmentsHelper.buildResponseData(Errors.NONE.code(), 0, errors);
+        ClientResponse response = new ClientResponse(null, null, null,
+                0L, 0L, false, false, null, null,
+                new AssignReplicasToDirsResponse(responseData));
+        return response;
+    }
+
+    @Test
+    public void testAssignmentCompaction() throws Exception {
+        // Delay the first controller response to force assignment compaction 
logic
+        CompletableFuture<Runnable> completionFuture = new 
CompletableFuture<>();
+        doAnswer(invocation -> {
+            AssignReplicasToDirsRequestData request = 
invocation.getArgument(0, 
AssignReplicasToDirsRequest.Builder.class).build().data();
+            ControllerRequestCompletionHandler completionHandler = 
invocation.getArgument(1, ControllerRequestCompletionHandler.class);
+            ClientResponse response = buildSuccessfulResponse(request);
+            Runnable completion = () -> completionHandler.onComplete(response);
+            if (completionFuture.isDone()) completion.run();
+            else completionFuture.complete(completion);
+            return null;
+        
}).when(channelManager).sendRequest(any(AssignReplicasToDirsRequest.Builder.class),
+                any(ControllerRequestCompletionHandler.class));
+
+        CountDownLatch remainingInvocations = new CountDownLatch(20);
+        Runnable onComplete = () -> {
+            assertTrue(completionFuture.isDone(), "Premature invocation");
+            assertTrue(remainingInvocations.getCount() > 0, "Extra 
invocation");
+            remainingInvocations.countDown();
+        };
+        Uuid[] dirs = {DIR_1, DIR_2, DIR_3};
+        for (int i = 0; i < remainingInvocations.getCount(); i++) {
+            time.sleep(100);
+            manager.onAssignment(new TopicIdPartition(TOPIC_1, 0), dirs[i % 
3], onComplete);
+        }
+        activeWait(completionFuture::isDone);
+        completionFuture.get().run();
+        activeWait(() -> remainingInvocations.getCount() == 0);
+    }
+
+    void activeWait(Supplier<Boolean> predicate) {
+        while (!predicate.get()) {
+            time.sleep(100);
+            manager.wakeup();
+            Thread.yield();
+        }
+    }

Review Comment:
   This infinite loop will cause test hanging forever if there's some bugs. 
Could we improve it? Same as L298. 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