jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224568806


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java:
##########
@@ -0,0 +1,803 @@
+/*
+ * 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 org.apache.kafka.coordinator.group.runtime;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashSet;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.apache.kafka.test.TestUtils.assertFutureThrows;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class CoordinatorRuntimeTest {
+    private static final TopicPartition TP = new 
TopicPartition("__consumer_offsets", 0);
+
+    /**
+     * An CoordinatorEventProcessor that directly executes the operations. 
This is
+     * useful in unit tests where execution in threads is not required.
+     */
+    private static class MockEventProcessor implements 
CoordinatorEventProcessor {
+        @Override
+        public void enqueue(CoordinatorEvent event) throws 
RejectedExecutionException {
+            try {
+                event.run();
+            } catch (Throwable ex) {
+                event.complete(ex);
+            }
+        }
+
+        @Override
+        public void close() throws Exception {}
+    }
+
+    /**
+     * A CoordinatorLoader that always succeeds.
+     */
+    private static class MockCoordinatorLoader implements 
CoordinatorLoader<String> {
+        @Override
+        public CompletableFuture<Void> load(TopicPartition tp, 
CoordinatorPlayback<String> replayable) {
+            return CompletableFuture.completedFuture(null);
+        }
+    }
+
+    /**
+     * An in-memory partition writer that accepts a maximum number of writes.
+     */
+    private static class MockPartitionWriter extends 
InMemoryPartitionWriter<String> {
+        private int allowedWrites = 1;
+
+        public MockPartitionWriter() {
+            this(Integer.MAX_VALUE);
+        }
+
+        public MockPartitionWriter(int allowedWrites) {
+            super(false);
+            this.allowedWrites = allowedWrites;
+        }
+
+        @Override
+        public void registerListener(TopicPartition tp, Listener listener) {
+            super.registerListener(tp, listener);
+        }
+
+        @Override
+        public void deregisterListener(TopicPartition tp, Listener listener) {
+            super.deregisterListener(tp, listener);
+        }
+
+        @Override
+        public long append(TopicPartition tp, List<String> records) throws 
KafkaException {
+            if (allowedWrites-- > 0) {
+                return super.append(tp, records);
+            } else {
+                throw new KafkaException("append failed.");
+            }
+        }
+    }
+
+    /**
+     * A simple Coordinator implementation that stores the records into a set.
+     */
+    private static class MockCoordinator implements Coordinator<String> {
+        private final TimelineHashSet<String> records;
+
+        MockCoordinator(
+            SnapshotRegistry snapshotRegistry
+        ) {
+            records = new TimelineHashSet<>(snapshotRegistry, 0);
+        }
+
+        @Override
+        public void replay(String record) throws RuntimeException {
+            records.add(record);
+        }
+
+        Set<String> records() {
+            return Collections.unmodifiableSet(new HashSet<>(records));
+        }
+    }
+
+    /**
+     * A CoordinatorBuilder that creates a MockCoordinator.
+     */
+    private static class MockCoordinatorBuilder implements 
CoordinatorBuilder<MockCoordinator, String> {
+        private SnapshotRegistry snapshotRegistry;
+
+        @Override
+        public CoordinatorBuilder<MockCoordinator, String> 
withSnapshotRegistry(
+            SnapshotRegistry snapshotRegistry
+        ) {
+            this.snapshotRegistry = snapshotRegistry;
+            return this;
+        }
+
+        @Override
+        public MockCoordinator build() {
+            return new 
MockCoordinator(Objects.requireNonNull(this.snapshotRegistry));
+        }
+    }
+
+    /**
+     * A CoordinatorBuilderSupplier that returns a MockCoordinatorBuilder.
+     */
+    private static class MockCoordinatorBuilderSupplier implements 
CoordinatorBuilderSupplier<MockCoordinator, String> {
+        @Override
+        public CoordinatorBuilder<MockCoordinator, String> get() {
+            return new MockCoordinatorBuilder();
+        }
+    }
+
+    @Test
+    public void testScheduleLoading() {
+        MockCoordinatorLoader loader = mock(MockCoordinatorLoader.class);
+        MockPartitionWriter writer = mock(MockPartitionWriter.class);
+        MockCoordinatorBuilderSupplier supplier = 
mock(MockCoordinatorBuilderSupplier.class);
+        MockCoordinatorBuilder builder = mock(MockCoordinatorBuilder.class);
+        MockCoordinator coordinator = mock(MockCoordinator.class);
+
+        CoordinatorRuntime<MockCoordinator, String> runtime =
+            new CoordinatorRuntime.Builder<MockCoordinator, String>()
+                .withLoader(loader)
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(writer)
+                .withCoordinatorBuilderSupplier(supplier)
+                .build();
+
+        when(builder.withSnapshotRegistry(any())).thenReturn(builder);
+        when(builder.build()).thenReturn(coordinator);
+        when(supplier.get()).thenReturn(builder);
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        when(loader.load(TP, coordinator)).thenReturn(future);
+
+        // Getting the coordinator context fails because the coordinator

Review Comment:
   Got it -- the comment is a little unclear, since we can get the context 
while it is still not fully loaded, but the issue is that we didn't start 
loading (which starts with creating the context). 
   
   Maybe we could say "Getting the coordinator context fails because the 
coordinator hasn't started loading and hasn't crated the context"? That might 
be a bit wordy, but something like that.



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