OmniaGM commented on code in PR #14369:
URL: https://github.com/apache/kafka/pull/14369#discussion_r1392653351


##########
core/src/test/java/kafka/server/AssignmentsManagerTest.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.AuthenticationException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.AssignReplicasToDirsRequestData;
+import org.apache.kafka.common.message.AssignReplicasToDirsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AssignReplicasToDirsRequest;
+import org.apache.kafka.common.requests.AssignReplicasToDirsResponse;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.server.common.TopicIdPartition;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+public class AssignmentsManagerTest {
+
+    private static final Uuid TOPIC_1 = 
Uuid.fromString("88rnFIqYSZykX4ZSKv81bg");
+    private static final Uuid TOPIC_2 = 
Uuid.fromString("VKCnzHdhR5uDQc1shqBYrQ");
+    private static final Uuid DIR_1 = 
Uuid.fromString("cbgD8WdLQCyzLrFIMBhv3w");
+    private static final Uuid DIR_2 = 
Uuid.fromString("zO0bDc0vSuam7Db9iH7rYQ");
+    private static final Uuid DIR_3 = 
Uuid.fromString("CGBWbrFkRkeJQy6Aryzq2Q");
+
+    private MockTime time;
+    private NodeToControllerChannelManager channelManager;
+    private AssignmentsManager manager;
+
+    @BeforeEach
+    public void setup() {
+        time = new MockTime();
+        channelManager = mock(NodeToControllerChannelManager.class);
+        manager = new AssignmentsManager(time, channelManager, 8, () -> 100L);
+    }
+
+    @AfterEach
+    void tearDown() throws InterruptedException {
+        manager.close();
+    }
+
+    @Test
+    void testBuildRequestData() {
+        Map<TopicIdPartition, Uuid> assignment = new HashMap<TopicIdPartition, 
Uuid>() {{
+                put(new TopicIdPartition(TOPIC_1, 1), DIR_1);
+                put(new TopicIdPartition(TOPIC_1, 2), DIR_2);
+                put(new TopicIdPartition(TOPIC_1, 3), DIR_3);
+                put(new TopicIdPartition(TOPIC_1, 4), DIR_1);
+                put(new TopicIdPartition(TOPIC_2, 5), DIR_2);
+            }};
+        AssignReplicasToDirsRequestData built = 
AssignmentsManager.buildRequestData(8, 100L, assignment);
+        AssignReplicasToDirsRequestData expected = new 
AssignReplicasToDirsRequestData()
+                .setBrokerId(8)
+                .setBrokerEpoch(100L)
+                .setDirectories(Arrays.asList(
+                        new AssignReplicasToDirsRequestData.DirectoryData()
+                                .setId(DIR_2)
+                                .setTopics(Arrays.asList(
+                                        new 
AssignReplicasToDirsRequestData.TopicData()
+                                                .setTopicId(TOPIC_1)
+                                                
.setPartitions(Collections.singletonList(
+                                                        new 
AssignReplicasToDirsRequestData.PartitionData()
+                                                                
.setPartitionIndex(2)
+                                                )),
+                                        new 
AssignReplicasToDirsRequestData.TopicData()
+                                                .setTopicId(TOPIC_2)
+                                                
.setPartitions(Collections.singletonList(
+                                                        new 
AssignReplicasToDirsRequestData.PartitionData()
+                                                                
.setPartitionIndex(5)
+                                                ))
+                                )),
+                        new AssignReplicasToDirsRequestData.DirectoryData()
+                                .setId(DIR_3)
+                                .setTopics(Collections.singletonList(
+                                        new 
AssignReplicasToDirsRequestData.TopicData()
+                                                .setTopicId(TOPIC_1)
+                                                
.setPartitions(Collections.singletonList(
+                                                        new 
AssignReplicasToDirsRequestData.PartitionData()
+                                                                
.setPartitionIndex(3)
+                                                ))
+                                )),
+                        new AssignReplicasToDirsRequestData.DirectoryData()
+                                .setId(DIR_1)
+                                .setTopics(Collections.singletonList(
+                                        new 
AssignReplicasToDirsRequestData.TopicData()
+                                                .setTopicId(TOPIC_1)
+                                                .setPartitions(Arrays.asList(
+                                                        new 
AssignReplicasToDirsRequestData.PartitionData()
+                                                                
.setPartitionIndex(4),
+                                                        new 
AssignReplicasToDirsRequestData.PartitionData()
+                                                                
.setPartitionIndex(1)
+                                                ))
+                                ))
+                ));
+        assertEquals(expected, built);
+    }
+
+    @Test
+    public void testAssignmentAggregation() throws InterruptedException {
+        CountDownLatch readyToAssert = new CountDownLatch(1);
+        doAnswer(invocation -> {
+            if (readyToAssert.getCount() > 0) {
+                readyToAssert.countDown();
+            }
+            return null;
+        
}).when(channelManager).sendRequest(any(AssignReplicasToDirsRequest.Builder.class),
 any(ControllerRequestCompletionHandler.class));
+
+        manager.onAssignment(new TopicIdPartition(TOPIC_1, 1), DIR_1);
+        manager.onAssignment(new TopicIdPartition(TOPIC_1, 2), DIR_2);
+        manager.onAssignment(new TopicIdPartition(TOPIC_1, 3), DIR_3);
+        manager.onAssignment(new TopicIdPartition(TOPIC_1, 4), DIR_1);
+        manager.onAssignment(new TopicIdPartition(TOPIC_2, 5), DIR_2);
+        while (!readyToAssert.await(1, TimeUnit.MILLISECONDS)) {
+            time.sleep(100);
+            manager.wakeup();
+        }
+
+        ArgumentCaptor<AssignReplicasToDirsRequest.Builder> captor = 
ArgumentCaptor.forClass(AssignReplicasToDirsRequest.Builder.class);
+        verify(channelManager).sendRequest(captor.capture(), 
any(ControllerRequestCompletionHandler.class));
+        verifyNoMoreInteractions(channelManager);
+        assertEquals(1, captor.getAllValues().size());
+        AssignReplicasToDirsRequestData actual = 
captor.getValue().build().data();
+        AssignReplicasToDirsRequestData expected = 
AssignmentsManager.buildRequestData(
+                8, 100L, new HashMap<TopicIdPartition, Uuid>() {{
+                        put(new TopicIdPartition(TOPIC_1, 1), DIR_1);
+                        put(new TopicIdPartition(TOPIC_1, 2), DIR_2);
+                        put(new TopicIdPartition(TOPIC_1, 3), DIR_3);
+                        put(new TopicIdPartition(TOPIC_1, 4), DIR_1);
+                        put(new TopicIdPartition(TOPIC_2, 5), DIR_2);
+                    }}
+        );
+        assertEquals(expected, actual);

Review Comment:
   This assertion is failing due to a different order of `DirectoryData.topics` 
here 
https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14369/11/testReport/junit/kafka.server/AssignmentsManagerTest/Build___JDK_8_and_Scala_2_12___testAssignmentAggregation__/
 



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

Reply via email to