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


##########
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
+        // is not loaded yet.
+        assertThrows(NotCoordinatorException.class, () -> 
runtime.contextOrThrow(TP));
+
+        // Schedule the loading.
+        runtime.scheduleLoadOperation(TP, 0);
+
+        // Getting the coordinator context succeeds now.
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+
+        // The coordinator is loading.
+        assertEquals(CoordinatorRuntime.CoordinatorState.LOADING, ctx.state);
+        assertEquals(0, ctx.epoch);
+        assertEquals(coordinator, ctx.coordinator);
+
+        // When the loading completes, the coordinator transitions to active.
+        future.complete(null);
+        assertEquals(CoordinatorRuntime.CoordinatorState.ACTIVE, ctx.state);
+
+        // Verify that onLoaded is called.
+        verify(coordinator, times(1)).onLoaded();
+
+        // Verify that the listener is registered.
+        verify(writer, times(1)).registerListener(
+            eq(TP),
+            any(PartitionWriter.Listener.class)
+        );
+    }
+
+    @Test
+    public void testScheduleLoadingWithFailure() {
+        MockCoordinatorLoader loader = mock(MockCoordinatorLoader.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(new MockPartitionWriter())
+                .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);
+
+        // Schedule the loading.
+        runtime.scheduleLoadOperation(TP, 0);
+
+        // Getting the context succeeds and the coordinator should be in 
loading.
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(CoordinatorRuntime.CoordinatorState.LOADING, ctx.state);
+        assertEquals(0, ctx.epoch);
+        assertEquals(coordinator, ctx.coordinator);
+
+        // When the loading fails, the coordinator transitions to failed.
+        future.completeExceptionally(new Exception("failure"));
+        assertEquals(CoordinatorRuntime.CoordinatorState.FAILED, ctx.state);
+
+        // Verify that onUnloaded is called.
+        verify(coordinator, times(1)).onUnloaded();
+    }
+
+    @Test
+    public void testScheduleLoadingWithStalePartitionEpoch() {
+        MockCoordinatorLoader loader = mock(MockCoordinatorLoader.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(new MockPartitionWriter())
+                .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);
+
+        // Schedule the loading.
+        runtime.scheduleLoadOperation(TP, 10);
+
+        // Getting the context succeeds and the coordinator should be in 
loading.
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(CoordinatorRuntime.CoordinatorState.LOADING, ctx.state);
+        assertEquals(10, ctx.epoch);
+        assertEquals(coordinator, ctx.coordinator);
+
+        // When the loading completes, the coordinator transitions to active.
+        future.complete(null);
+        assertEquals(CoordinatorRuntime.CoordinatorState.ACTIVE, ctx.state);
+        assertEquals(10, ctx.epoch);
+
+        // Loading with a previous epoch is a no-op. The coordinator stays
+        // in active state with the correct epoch.
+        runtime.scheduleLoadOperation(TP, 0);
+        assertEquals(CoordinatorRuntime.CoordinatorState.ACTIVE, ctx.state);
+        assertEquals(10, ctx.epoch);
+    }
+
+    @Test
+    public void testScheduleLoadingAfterLoadingFailure() {
+        MockCoordinatorLoader loader = mock(MockCoordinatorLoader.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(new MockPartitionWriter())
+                .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);
+
+        // Schedule the loading.
+        runtime.scheduleLoadOperation(TP, 10);
+
+        // Getting the context succeeds and the coordinator should be in 
loading.
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(CoordinatorRuntime.CoordinatorState.LOADING, ctx.state);
+        assertEquals(10, ctx.epoch);
+        assertEquals(coordinator, ctx.coordinator);
+
+        // When the loading fails, the coordinator transitions to failed.
+        future.completeExceptionally(new Exception("failure"));
+        assertEquals(CoordinatorRuntime.CoordinatorState.FAILED, ctx.state);
+
+        // Verify that onUnloaded is called.
+        verify(coordinator, times(1)).onUnloaded();
+
+        // Create a new coordinator.
+        coordinator = mock(MockCoordinator.class);
+        when(builder.build()).thenReturn(coordinator);
+
+        // Schedule the reloading.
+        future = new CompletableFuture<>();
+        when(loader.load(TP, coordinator)).thenReturn(future);
+        runtime.scheduleLoadOperation(TP, 11);
+
+        // Getting the context succeeds and the coordinator should be in 
loading.
+        ctx = runtime.contextOrThrow(TP);
+        assertEquals(CoordinatorRuntime.CoordinatorState.LOADING, ctx.state);
+        assertEquals(11, ctx.epoch);
+        assertEquals(coordinator, ctx.coordinator);
+
+        // Complete the loading.
+        future.complete(null);
+
+        // Verify the state.
+        assertEquals(CoordinatorRuntime.CoordinatorState.ACTIVE, ctx.state);
+    }
+
+    @Test
+    public void testScheduleUnloading() {
+        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(new MockCoordinatorLoader())
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(writer)
+                .withCoordinatorBuilderSupplier(supplier)
+                .build();
+
+        when(builder.withSnapshotRegistry(any())).thenReturn(builder);
+        when(builder.build()).thenReturn(coordinator);
+        when(supplier.get()).thenReturn(builder);
+
+        // Loads the coordinator. It directly transitions to active.
+        runtime.scheduleLoadOperation(TP, 10);
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(CoordinatorRuntime.CoordinatorState.ACTIVE, ctx.state);
+        assertEquals(10, ctx.epoch);
+
+        // Schedule the unloading.
+        runtime.scheduleUnloadOperation(TP, ctx.epoch + 1);
+
+        // Verify that onUnloaded is called.
+        verify(coordinator, times(1)).onUnloaded();
+
+        // Verify that the listener is deregistered.
+        verify(writer, times(1)).deregisterListener(
+            eq(TP),
+            any(PartitionWriter.Listener.class)
+        );
+
+        // Getting the coordinator context fails because it no longer exists.
+        assertThrows(NotCoordinatorException.class, () -> 
runtime.contextOrThrow(TP));
+    }
+
+    @Test
+    public void testScheduleUnloadingWithStalePartitionEpoch() {
+        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(new MockCoordinatorLoader())
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(new MockPartitionWriter())
+                .withCoordinatorBuilderSupplier(supplier)
+                .build();
+
+        when(builder.withSnapshotRegistry(any())).thenReturn(builder);
+        when(builder.build()).thenReturn(coordinator);
+        when(supplier.get()).thenReturn(builder);
+
+        // Loads the coordinator. It directly transitions to active.
+        runtime.scheduleLoadOperation(TP, 10);
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(CoordinatorRuntime.CoordinatorState.ACTIVE, ctx.state);
+        assertEquals(10, ctx.epoch);
+
+        // Unloading with a previous epoch is a no-op. The coordinator stays
+        // in active with the correct epoch.
+        runtime.scheduleUnloadOperation(TP, 0);
+        assertEquals(CoordinatorRuntime.CoordinatorState.ACTIVE, ctx.state);
+        assertEquals(10, ctx.epoch);
+    }
+
+    @Test
+    public void testScheduleWriteOp() throws ExecutionException, 
InterruptedException, TimeoutException {
+        MockPartitionWriter writer = new MockPartitionWriter();
+
+        CoordinatorRuntime<MockCoordinator, String> runtime =
+            new CoordinatorRuntime.Builder<MockCoordinator, String>()
+                .withLoader(new MockCoordinatorLoader())
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(writer)
+                .withCoordinatorBuilderSupplier(new 
MockCoordinatorBuilderSupplier())
+                .build();
+
+        // Schedule the loading.
+        runtime.scheduleLoadOperation(TP, 10);
+
+        // Verify the initial state.
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(0L, ctx.lastWrittenOffset);
+        assertEquals(0L, ctx.lastCommittedOffset);
+        assertEquals(Collections.singletonList(0L), 
ctx.snapshotRegistry.epochsList());
+
+        // Write #1.
+        CompletableFuture<String> write1 = 
runtime.scheduleWriteOperation("write#1", TP,
+            state -> new CoordinatorResult<>(Arrays.asList("record1", 
"record2"), "response1"));
+
+        // Verify that the write is not completed yet.
+        assertFalse(write1.isDone());
+
+        // The last written offset is updated.
+        assertEquals(2L, ctx.lastWrittenOffset);
+        // The last committed offset does not change.
+        assertEquals(0L, ctx.lastCommittedOffset);
+        // A new snapshot is created.
+        assertEquals(Arrays.asList(0L, 2L), ctx.snapshotRegistry.epochsList());
+        // Records have been replayed to the coordinator.
+        assertEquals(mkSet("record1", "record2"), ctx.coordinator.records());
+        // Records have been written to the log.
+        assertEquals(Arrays.asList("record1", "record2"), writer.records(TP));
+
+        // Write #2.
+        CompletableFuture<String> write2 = 
runtime.scheduleWriteOperation("write#2", TP,
+            state -> new CoordinatorResult<>(Arrays.asList("record3"), 
"response2"));
+
+        // Verify that the write is not completed yet.
+        assertFalse(write2.isDone());
+
+        // The last written offset is updated.
+        assertEquals(3L, ctx.lastWrittenOffset);
+        // The last committed offset does not change.
+        assertEquals(0L, ctx.lastCommittedOffset);
+        // A new snapshot is created.
+        assertEquals(Arrays.asList(0L, 2L, 3L), 
ctx.snapshotRegistry.epochsList());
+        // Records have been replayed to the coordinator.
+        assertEquals(mkSet("record1", "record2", "record3"), 
ctx.coordinator.records());
+        // Records have been written to the log.
+        assertEquals(Arrays.asList("record1", "record2", "record3"), 
writer.records(TP));
+
+        // Write #3 but without any records.
+        CompletableFuture<String> write3 = 
runtime.scheduleWriteOperation("write#3", TP,
+            state -> new CoordinatorResult<>(Collections.emptyList(), 
"response3"));
+
+        // Verify that the write is not completed yet.
+        assertFalse(write3.isDone());
+
+        // The state does not change.
+        assertEquals(3L, ctx.lastWrittenOffset);
+        assertEquals(0L, ctx.lastCommittedOffset);
+        assertEquals(Arrays.asList(0L, 2L, 3L), 
ctx.snapshotRegistry.epochsList());
+        assertEquals(mkSet("record1", "record2", "record3"), 
ctx.coordinator.records());
+        assertEquals(Arrays.asList("record1", "record2", "record3"), 
writer.records(TP));
+
+        // Commit write #1.
+        writer.commit(TP, 2);
+
+        // The write is completed.
+        assertTrue(write1.isDone());
+        assertEquals("response1", write1.get(5, TimeUnit.SECONDS));
+
+        // The last committed offset is updated.
+        assertEquals(2L, ctx.lastCommittedOffset);
+        // The snapshot is cleaned up.
+        assertEquals(Arrays.asList(2L, 3L), ctx.snapshotRegistry.epochsList());
+
+        // Commit write #2.
+        writer.commit(TP, 3);
+
+        // The writes are completed.
+        assertTrue(write2.isDone());
+        assertTrue(write3.isDone());
+        assertEquals("response2", write2.get(5, TimeUnit.SECONDS));
+        assertEquals("response3", write3.get(5, TimeUnit.SECONDS));
+
+        // The last committed offset is updated.
+        assertEquals(3L, ctx.lastCommittedOffset);
+        // The snapshot is cleaned up.
+        assertEquals(Collections.singletonList(3L), 
ctx.snapshotRegistry.epochsList());
+
+        // Write #4 but without records.
+        CompletableFuture<String> write4 = 
runtime.scheduleWriteOperation("write#4", TP,
+            state -> new CoordinatorResult<>(Collections.emptyList(), 
"response4"));
+
+        // It is completed immediately because the state is fully commited.
+        assertTrue(write4.isDone());
+        assertEquals("response4", write4.get(5, TimeUnit.SECONDS));
+    }
+
+    @Test
+    public void testScheduleWriteOpWhenInactive() {
+        CoordinatorRuntime<MockCoordinator, String> runtime =
+            new CoordinatorRuntime.Builder<MockCoordinator, String>()
+                .withLoader(new MockCoordinatorLoader())
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(new MockPartitionWriter())
+                .withCoordinatorBuilderSupplier(new 
MockCoordinatorBuilderSupplier())
+                .build();
+
+        // Scheduling a write fails with a NotCoordinatorException because the 
coordinator
+        // does not exist.
+        CompletableFuture<String> write = 
runtime.scheduleWriteOperation("write", TP,
+            state -> new CoordinatorResult<>(Collections.emptyList(), 
"response1"));
+        assertFutureThrows(write, NotCoordinatorException.class);
+    }
+
+    @Test
+    public void testScheduleWriteOpWhenOpFails() {
+        CoordinatorRuntime<MockCoordinator, String> runtime =
+            new CoordinatorRuntime.Builder<MockCoordinator, String>()
+                .withLoader(new MockCoordinatorLoader())
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(new MockPartitionWriter())
+                .withCoordinatorBuilderSupplier(new 
MockCoordinatorBuilderSupplier())
+                .build();
+
+        // Loads the coordinator.
+        runtime.scheduleLoadOperation(TP, 10);
+
+        // Scheduling a write that fails when the operation is called. The 
exception
+        // is used to complete the future.
+        CompletableFuture<String> write = 
runtime.scheduleWriteOperation("write", TP, state -> {
+            throw new KafkaException("error");
+        });
+        assertFutureThrows(write, KafkaException.class);
+    }
+
+    @Test
+    public void testScheduleWriteOpWhenReplayFails() {
+        CoordinatorRuntime<MockCoordinator, String> runtime =
+            new CoordinatorRuntime.Builder<MockCoordinator, String>()
+                .withLoader(new MockCoordinatorLoader())
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(new MockPartitionWriter())
+                .withCoordinatorBuilderSupplier(new 
MockCoordinatorBuilderSupplier())
+                .build();
+
+        // Loads the coordinator.
+        runtime.scheduleLoadOperation(TP, 10);
+
+        // Verify the initial state.
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(0L, ctx.lastWrittenOffset);
+        assertEquals(0L, ctx.lastCommittedOffset);
+        assertEquals(Collections.singletonList(0L), 
ctx.snapshotRegistry.epochsList());
+
+        // Override the coordinator with a coordinator that throws
+        // an exception when replay is called.
+        ctx.coordinator = new MockCoordinator(ctx.snapshotRegistry) {
+            @Override
+            public void replay(String record) throws RuntimeException {
+                throw new IllegalArgumentException("error");
+            }
+        };
+
+        // Write. It should fail.
+        CompletableFuture<String> write = 
runtime.scheduleWriteOperation("write", TP,
+            state -> new CoordinatorResult<>(Arrays.asList("record1", 
"record2"), "response1"));
+        assertFutureThrows(write, IllegalArgumentException.class);
+
+        // Verify that the state has not changed.
+        assertEquals(0L, ctx.lastWrittenOffset);
+        assertEquals(0L, ctx.lastCommittedOffset);
+        assertEquals(Collections.singletonList(0L), 
ctx.snapshotRegistry.epochsList());
+    }
+
+    @Test
+    public void testScheduleWriteOpWhenWriteFails() {
+        // The partition writer only accept on write.
+        MockPartitionWriter writer = new MockPartitionWriter(1);
+
+        CoordinatorRuntime<MockCoordinator, String> runtime =
+            new CoordinatorRuntime.Builder<MockCoordinator, String>()
+                .withLoader(new MockCoordinatorLoader())
+                .withEventProcessor(new MockEventProcessor())
+                .withPartitionWriter(writer)
+                .withCoordinatorBuilderSupplier(new 
MockCoordinatorBuilderSupplier())
+                .build();
+
+        // Loads the coordinator.
+        runtime.scheduleLoadOperation(TP, 10);
+
+        // Verify the initial state.
+        CoordinatorRuntime<MockCoordinator, String>.CoordinatorContext ctx = 
runtime.contextOrThrow(TP);
+        assertEquals(0, ctx.lastWrittenOffset);
+        assertEquals(0, ctx.lastCommittedOffset);
+        assertEquals(Collections.singletonList(0L), 
ctx.snapshotRegistry.epochsList());
+
+        // Write #1. It should succeed and be applied to the coordinator.
+        runtime.scheduleWriteOperation("write#1", TP,
+            state -> new CoordinatorResult<>(Arrays.asList("record1", 
"record2"), "response1"));
+
+        // Verify that the state has been updated.
+        assertEquals(2L, ctx.lastWrittenOffset);
+        assertEquals(0L, ctx.lastCommittedOffset);
+        assertEquals(Arrays.asList(0L, 2L), ctx.snapshotRegistry.epochsList());
+        assertEquals(mkSet("record1", "record2"), ctx.coordinator.records());
+
+        // Write #2. It should fail.

Review Comment:
   This fails because we only allow one write? (That took me a moment to 
realize)
   
   If we returned 3 records in write2, I think this test would be stronger.



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