cmccabe commented on code in PR #16672:
URL: https://github.com/apache/kafka/pull/16672#discussion_r1695816959


##########
server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java:
##########
@@ -14,451 +14,506 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.kafka.server;
 
+import com.yammer.metrics.core.Gauge;
+import com.yammer.metrics.core.Metric;
+import com.yammer.metrics.core.MetricName;
+import com.yammer.metrics.core.MetricsRegistry;
 import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.NodeApiVersions;
 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.ApiVersionsResponseData;
 import org.apache.kafka.common.message.AssignReplicasToDirsRequestData;
 import org.apache.kafka.common.message.AssignReplicasToDirsResponseData;
+import org.apache.kafka.common.metadata.FeatureLevelRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.ApiVersionsResponse;
 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.metadata.AssignmentsHelper;
+import org.apache.kafka.common.utils.ExponentialBackoff;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.MetadataProvenance;
+import org.apache.kafka.server.common.MetadataVersion;
 import org.apache.kafka.server.common.TopicIdPartition;
-import org.apache.kafka.server.metrics.KafkaYammerMetrics;
 import org.apache.kafka.test.TestUtils;
 
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Metric;
-import com.yammer.metrics.core.MetricName;
-
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
-import org.mockito.ArgumentCaptor;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.LinkedBlockingDeque;
 import java.util.function.Function;
-import java.util.function.Supplier;
 
-import static org.apache.kafka.metadata.AssignmentsHelper.buildRequestData;
+import static 
org.apache.kafka.server.AssignmentsManager.QUEUED_REPLICA_TO_DIR_ASSIGNMENTS_METRIC;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.atMostOnce;
-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 Logger LOG = 
LoggerFactory.getLogger(AssignmentsManagerTest.class);
     private static final Uuid TOPIC_1 = 
Uuid.fromString("88rnFIqYSZykX4ZSKv81bg");
     private static final Uuid TOPIC_2 = 
Uuid.fromString("VKCnzHdhR5uDQc1shqBYrQ");
+    private static final Uuid TOPIC_3 = 
Uuid.fromString("ZeAwvYt-Ro2suQudGUdbRg");
     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);
-        Map<Uuid, String> topicNames = new HashMap<>();
-        topicNames.put(TOPIC_1, "TOPIC_1");
-        topicNames.put(TOPIC_2, "TOPIC_2");
-        Map<Uuid, String> dirPaths = new HashMap<>();
-        dirPaths.put(DIR_1, "DIR_1");
-        dirPaths.put(DIR_2, "DIR_2");
-        dirPaths.put(DIR_3, "DIR_3");
-        manager = new AssignmentsManager(time, channelManager, 8, () -> 100L,
-                id -> Optional.ofNullable(dirPaths.get(id)), id -> 
Optional.ofNullable(topicNames.get(id)));
+    private static final MetadataImage TEST_IMAGE;
+
+    static {
+        MetadataDelta delta = new MetadataDelta.Builder().
+            setImage(MetadataImage.EMPTY).
+            build();
+        delta.replay(new FeatureLevelRecord().
+            setName(MetadataVersion.FEATURE_NAME).
+            setFeatureLevel(MetadataVersion.IBP_3_8_IV0.featureLevel()));
+        delta.replay(new RegisterBrokerRecord().
+            setBrokerId(0).
+            setIncarnationId(Uuid.fromString("JJsH6zB0R7eKbr0Sy49ULw")).
+            setBrokerEpoch(123));
+        delta.replay(new RegisterBrokerRecord().
+            setBrokerId(1).
+            setIncarnationId(Uuid.fromString("DtnWclXyQ4qNDvL97JlnvQ")).
+            setBrokerEpoch(456));
+        delta.replay(new RegisterBrokerRecord().
+            setBrokerId(2).
+            setIncarnationId(Uuid.fromString("UFa_RKgLR4mxEXyquEPEmg")).
+            setBrokerEpoch(789));
+        delta.replay(new RegisterBrokerRecord().
+            setBrokerId(3).
+            setIncarnationId(Uuid.fromString("jj-cnHYASAmb_H9JR6nmtQ")).
+            setBrokerEpoch(987));
+        delta.replay(new TopicRecord().
+            setName("foo").
+            setTopicId(TOPIC_1));
+        delta.replay(new PartitionRecord().
+            setPartitionId(0).
+            setTopicId(TOPIC_1).
+            setReplicas(Arrays.asList(0, 1, 2)).
+            setIsr(Arrays.asList(0, 1, 2)).
+            setLeader(1));
+        delta.replay(new PartitionRecord().
+            setPartitionId(1).
+            setTopicId(TOPIC_1).
+            setReplicas(Arrays.asList(1, 2, 3)).
+            setIsr(Arrays.asList(1, 2, 3)).
+            setLeader(1));
+        delta.replay(new TopicRecord().
+            setName("bar").
+            setTopicId(TOPIC_2));
+        delta.replay(new PartitionRecord().
+            setPartitionId(0).
+            setTopicId(TOPIC_2).
+            setReplicas(Arrays.asList(0, 3, 2)).
+            setIsr(Arrays.asList(0, 3, 2)).
+            setLeader(1));
+        delta.replay(new PartitionRecord().
+            setPartitionId(1).
+            setTopicId(TOPIC_2).
+            setReplicas(Arrays.asList(1, 2, 3)).
+            setIsr(Arrays.asList(2)).
+            setLeader(2));
+        delta.replay(new PartitionRecord().
+            setPartitionId(2).
+            setTopicId(TOPIC_2).
+            setReplicas(Arrays.asList(3, 2, 1)).
+            setIsr(Arrays.asList(3, 2, 1)).
+            setLeader(3));
+        TEST_IMAGE = delta.apply(MetadataProvenance.EMPTY);
     }
 
-    @AfterEach
-    void tearDown() throws InterruptedException {
-        manager.close();
-    }
+    static class MockNodeToControllerChannelManager implements 
NodeToControllerChannelManager {
+        LinkedBlockingDeque<Map.Entry<AssignReplicasToDirsRequestData, 
ControllerRequestCompletionHandler>> callbacks =
+            new LinkedBlockingDeque<>();
 
-    AssignReplicasToDirsRequestData normalize(AssignReplicasToDirsRequestData 
request) {
-        request = request.duplicate();
-        request.directories().sort(Comparator.comparing(
-            AssignReplicasToDirsRequestData.DirectoryData::id));
-        for (AssignReplicasToDirsRequestData.DirectoryData directory : 
request.directories()) {
-            directory.topics().sort(Comparator.comparing(
-                AssignReplicasToDirsRequestData.TopicData::topicId));
-            for (AssignReplicasToDirsRequestData.TopicData topic : 
directory.topics()) {
-                topic.partitions().sort(Comparator.comparing(
-                    
AssignReplicasToDirsRequestData.PartitionData::partitionIndex));
-            }
+        @Override
+        public void start() {
         }
-        return request;
-    }
 
+        @Override
+        public void shutdown() {
+        }
 
-    void assertRequestEquals(
-        AssignReplicasToDirsRequestData expected,
-        AssignReplicasToDirsRequestData actual
-    ) {
-        assertEquals(normalize(expected), normalize(actual));
-    }
+        @Override
+        public Optional<NodeApiVersions> controllerApiVersions() {
+            return Optional.empty();
+        }
 
-    @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)))))));
-        assertRequestEquals(expected, built);
-    }
+        @Override
+        public void sendRequest(
+            AbstractRequest.Builder<? extends AbstractRequest> request,
+            ControllerRequestCompletionHandler callback
+        ) {
+            AssignReplicasToDirsRequest inputRequest = 
(AssignReplicasToDirsRequest) request.build();
+            synchronized (this) {
+                callbacks.add(new 
AbstractMap.SimpleEntry<>(inputRequest.data(), callback));
+            }
+        }
 
-    @Test
-    public void testAssignmentAggregation() throws InterruptedException {
-        CountDownLatch readyToAssert = new CountDownLatch(1);
-        doAnswer(invocation -> {
-            readyToAssert.countDown();
-            return null;
-        
}).when(channelManager).sendRequest(any(AssignReplicasToDirsRequest.Builder.class),
-            any(ControllerRequestCompletionHandler.class));
-
-        manager.onAssignment(new TopicIdPartition(TOPIC_1, 1), DIR_1, 
"testAssignmentAggregation", () -> { });
-        manager.onAssignment(new TopicIdPartition(TOPIC_1, 2), DIR_2, 
"testAssignmentAggregation", () -> { });
-        manager.onAssignment(new TopicIdPartition(TOPIC_1, 3), DIR_3, 
"testAssignmentAggregation", () -> { });
-        manager.onAssignment(new TopicIdPartition(TOPIC_1, 4), DIR_1, 
"testAssignmentAggregation", () -> { });
-        manager.onAssignment(new TopicIdPartition(TOPIC_2, 5), DIR_2, 
"testAssignmentAggregation", () -> { });
-        TestUtils.waitForCondition(() -> {
-            time.sleep(100);
-            manager.wakeup();
-            return readyToAssert.await(1, TimeUnit.MILLISECONDS);
-        }, "Timed out waiting for AssignReplicasToDirsRequest to be sent.");
-
-        ArgumentCaptor<AssignReplicasToDirsRequest.Builder> captor =
-            ArgumentCaptor.forClass(AssignReplicasToDirsRequest.Builder.class);
-        verify(channelManager, times(1)).start();
-        verify(channelManager).sendRequest(captor.capture(), 
any(ControllerRequestCompletionHandler.class));
-        verify(channelManager, atMostOnce()).shutdown();
-        verifyNoMoreInteractions(channelManager);
-        assertEquals(1, captor.getAllValues().size());
-        AssignReplicasToDirsRequestData actual = 
captor.getValue().build().data();
-        AssignReplicasToDirsRequestData expected = 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);
-                }}
-        );
-        assertRequestEquals(expected, actual);
+        void completeCallback(Function<AssignReplicasToDirsRequestData, 
Optional<ClientResponse>> completionist) throws InterruptedException {
+            Map.Entry<AssignReplicasToDirsRequestData, 
ControllerRequestCompletionHandler> entry = callbacks.take();
+            Optional<ClientResponse> clientResponse = 
completionist.apply(entry.getKey());
+            if (clientResponse.isPresent()) {
+                entry.getValue().onComplete(clientResponse.get());
+            } else {
+                entry.getValue().onTimeout();
+            }
+        }
     }
 
-    @Test
-    void testRequeuesFailedAssignmentPropagations() throws 
InterruptedException {
-        CountDownLatch readyToAssert = new CountDownLatch(5);
-        doAnswer(invocation -> {
-            readyToAssert.countDown();
-            if (readyToAssert.getCount() == 4) {
-                invocation.getArgument(1, 
ControllerRequestCompletionHandler.class).onTimeout();
-                manager.onAssignment(new TopicIdPartition(TOPIC_1, 2), DIR_3, 
"testRequeuesFailedAssignmentPropagations", () -> { });
+    static class TestEnv implements AutoCloseable {
+        final ExponentialBackoff backoff;
+        final MockNodeToControllerChannelManager channelManager;
+        final MetricsRegistry metricsRegistry = new MetricsRegistry();
+        final AssignmentsManager assignmentsManager;
+        final Map<TopicIdPartition, Integer> successes;
+
+        TestEnv() {
+            this.backoff = new ExponentialBackoff(1, 2, 4, 0);
+            this.channelManager = new MockNodeToControllerChannelManager();
+            this.assignmentsManager = new AssignmentsManager(
+                    backoff, Time.SYSTEM, channelManager, 1, () -> TEST_IMAGE, 
metricsRegistry);
+            this.successes = new HashMap<>();
+        }
+
+        void onAssignment(TopicIdPartition topicIdPartition, Uuid directoryId) 
{
+            assignmentsManager.onAssignment(topicIdPartition, directoryId, 
"test", () -> {
+                synchronized (successes) {
+                    successes.put(topicIdPartition, 
successes.getOrDefault(topicIdPartition, 0) + 1);
+                }
+            });
+        }
+
+        int success(TopicIdPartition topicIdPartition) {
+            synchronized (successes) {
+                return successes.getOrDefault(topicIdPartition, 0);
             }
-            if (readyToAssert.getCount() == 3) {
-                invocation.getArgument(1, 
ControllerRequestCompletionHandler.class).onComplete(
-                    new ClientResponse(null, null, null, 0L, 0L, false, false,
-                        new UnsupportedVersionException("test unsupported 
version exception"), null, null));
+        }
 
-                // duplicate should be ignored
-                manager.onAssignment(new TopicIdPartition(TOPIC_1, 2), DIR_3, 
"testRequeuesFailedAssignmentPropagations", () -> { });
+        void successfullyCompleteCallbackOfRequestAssigningTopic1ToDir1() 
throws Exception {
+            channelManager.completeCallback(req -> {
+                AssignReplicasToDirsRequestData.DirectoryData directoryData = 
req.directories().get(0);
+                assertEquals(DIR_1, directoryData.id());
+                AssignReplicasToDirsRequestData.TopicData topicData = 
directoryData.topics().get(0);
+                assertEquals(TOPIC_1, topicData.topicId());
+                assertEquals(0, 
topicData.partitions().get(0).partitionIndex());
+                return mockClientResponse(new 
AssignReplicasToDirsResponseData().
+                    setDirectories(Arrays.asList(new 
AssignReplicasToDirsResponseData.DirectoryData().
+                        setId(DIR_1).
+                        setTopics(Arrays.asList(new 
AssignReplicasToDirsResponseData.TopicData().
+                            setTopicId(TOPIC_1).
+                            setPartitions(Arrays.asList(new 
AssignReplicasToDirsResponseData.PartitionData().
+                                setPartitionIndex(0).
+                                setErrorCode((short) 0))))))));

Review Comment:
   hmm, I don't think that would be fewer lines since I'd have to create the 
nested maps



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