showuon commented on code in PR #15521: URL: https://github.com/apache/kafka/pull/15521#discussion_r1546047970
########## server/src/main/java/org/apache/kafka/server/AssignmentsManager.java: ########## @@ -192,24 +189,27 @@ void onComplete() { @Override public void run() throws Exception { AssignmentEvent existing = pending.getOrDefault(partition, null); + boolean existingIsInFlight = false; if (existing == null && inflight != null) { existing = inflight.getOrDefault(partition, null); + existingIsInFlight = true; } if (existing != null) { if (existing.dirId.equals(dirId)) { existing.merge(this); - if (log.isDebugEnabled()) log.debug("Ignoring duplicate assignment {}", this); + log.debug("Ignoring duplicate assignment {}", this); return; } if (existing.timestampNs > timestampNs) { - existing.onComplete(); - if (log.isDebugEnabled()) log.debug("Dropping assignment {} because it's older than {}", this, existing); + existing.merge(this); + log.debug("Dropping assignment {} because it's older than existing {}", this, existing); return; + } else if (!existingIsInFlight) { + this.merge(existing); + log.debug("Dropping existing assignment {} because it's older than {}", existing, this); } } - if (log.isDebugEnabled()) { - log.debug("Received new assignment {}", this); - } + log.debug("Received new assignment {}", this); Review Comment: Should we put this log on the top of `run` method? ########## 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: `TestUtils.waitForCondition` should be the one you needed. -- 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