You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "dajac (via GitHub)" <gi...@apache.org> on 2023/06/01 13:00:11 UTC

[GitHub] [kafka] dajac opened a new pull request, #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

dajac opened a new pull request, #13795:
URL: https://github.com/apache/kafka/pull/13795

   This PR includes temporary https://github.com/apache/kafka/pull/13666, https://github.com/apache/kafka/pull/13675, https://github.com/apache/kafka/pull/13793 and https://github.com/apache/kafka/pull/13794.
   
   This patch introduces the CoordinatorRuntime. The CoordinatorRuntime is a framework which encapsulate all the common features requires to build a coordinator such as the group coordinator. Please refer to the javadoc of that class for the details.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222177376


##########
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));

Review Comment:
   Will the snapshot registry have 4L as the epochList now?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1220267344


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,

Review Comment:
   Ok -- so it's cases where a request has the potential to write, but if no change will not. Makes sense.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224008603


##########
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());

Review Comment:
   Right. We always keep the last snapshot as we may have to rollback to it when a new write it applied but fails.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224034599


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);

Review Comment:
   I know. I actually prefer to have the exception's message included in the message logged. I mean on the same line because it is easier when you search the log. However, I could add a third argument to print the stack trace. Let me see if that makes sense here.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222035271


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorEvent.java:
##########
@@ -16,21 +16,23 @@
  */
 package org.apache.kafka.coordinator.group.runtime;
 
+import org.apache.kafka.common.TopicPartition;
+
 /**
  * The base event type used by all events processed in the
  * coordinator runtime.
  */
-public interface CoordinatorEvent extends EventAccumulator.Event<Integer> {
+public interface CoordinatorEvent extends EventAccumulator.Event<TopicPartition> {
 
     /**
-     * Runs the event.
+     * Executes the event.
      */
     void run();
 
     /**
      * Completes the event with the provided exception.
      *
-     * @param exception An exception to complete the event with.
+     * @param exception An exception if the processing of the event failed or null.

Review Comment:
   nit: maybe "An exception if the processing of the event failed or null otherwise"
   I read this as exception if the event failed or was null. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224005442


##########
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);

Review Comment:
   I do this to ensure that a new coordinator is created in this case. We verify the new value a few line below: `assertEquals(coordinator, ctx.coordinator);`.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1219010945


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {

Review Comment:
   > Would it make more sense to just call this in the finally block
   
   I am not sure to understand what you mean by this. Could you elaborate?
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1220268552


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.

Review Comment:
   It may be worth clarifying what "internal" means in the comment



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1226545970


##########
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:
   Updated the comment.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218619228


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            log.error("Execution of {} failed due to {}.", name, exception);
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new CoordinatorInternalEvent(name, tp, op));
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+
+                switch (context.state) {
+                    case INITIAL:
+                    case FAILED:
+                    case CLOSED:
+                        context.transitionTo(CoordinatorState.LOADING);
+                        loader.load(tp, context.coordinator).whenComplete((state, exception) -> {
+                            scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+                                CoordinatorContext ctx = contextOrThrow(tp);
+                                try {
+                                    if (exception != null) throw exception;
+                                    ctx.transitionTo(CoordinatorState.ACTIVE);
+                                    log.info("Finished loading of metadata from {} with epoch {}.", tp, partitionEpoch);
+                                } catch (Throwable ex) {
+                                    log.error("Failed to load metadata from {} due to {}.", tp, ex);
+                                    ctx.transitionTo(CoordinatorState.FAILED);
+                                }
+                            });
+                        });
+                        break;
+
+                    case LOADING:
+                        log.info("The coordinator {} is already loading metadata.", tp);
+                        break;
+
+                    case ACTIVE:
+                        log.info("The coordinator {} is already active.", tp);
+                        break;
+                }
+            } else {
+                log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.",
+                    context.tp, context.epoch, partitionEpoch);
+            }
+        });
+    }
+
+    /**
+     * Schedules the unloading of a coordinator. This is called when the broker is not the
+     * leader anymore.
+     *
+     * @param tp                The topic partition of the coordinator.
+     * @param partitionEpoch    The partition epoch.
+     */
+    public void scheduleUnloadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling unloading of metadata for {} with epoch {}", tp, partitionEpoch);
+
+        scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = contextOrThrow(tp);
+            if (context.epoch < partitionEpoch) {

Review Comment:
   Does the context.epoch stay the same until the next load?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224575011


##########
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:
   Yeah -- comments would help. I think the other part I was suggesting is that write #2 seems to be the same as write #1. So if it didn't fail due to the write limit, would we have in the records ["record1", "record2", "record1", "record2"]? If that's the case this is fine. 
   
   I originally thought that if the records were the same, it was a no op, but I was confusing things.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224569985


##########
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);

Review Comment:
   Ah I see. When I saw `assertEquals(coordinator, ctx.coordinator)` before I was confused because that would be true with the old coordinator. But we are testing that the ctx is updating. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1214859981


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp

Review Comment:
   does this mean a coordinator will be associated with multiple coordinator contexts since a coordinator owns multiple partitions?
   
   never mind -- looks like we have 1-1 mapping of coordinator to topic partition



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.

Review Comment:
   i'm not sure what the comment is saying here



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.

Review Comment:
   we can apply to the state machine first because we will revert to latest committed snapshot if the append fails right?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;

Review Comment:
   nit: i think it's a bit confusing to use `state` since we have CoordinatorState. can we use stateMachine?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.

Review Comment:
   should this be "internal event"?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class InternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        InternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            log.error("Execution of {} failed due to {}.", name, exception);
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                                The log context.
+     * @param processor                                 The event processor.
+     * @param partitionWriter                           The partition writer.
+     * @param loader                                    The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new InternalEvent(name, tp, op));
+    }
+
+    /**
+     * Builds a new coordinator state machine.
+     *
+     * @param context The coordinator context.
+     *
+     * @return The built coordinator state machine.
+     */
+    private S buildCoordinator(
+        CoordinatorContext context
+    ) {
+        return coordinatorBuilderSupplier
+            .get()
+            .withSnapshotRegistry(context.snapshotRegistry)
+            .build();
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+                if (context.state == CoordinatorState.LOADING) {
+                    log.info("Already loading metadata from {}.", tp);
+                } else {
+                    context.state = CoordinatorState.LOADING;
+                    context.coordinator = buildCoordinator(context);
+
+                    loader.load(
+                        tp,
+                        context.coordinator
+                    ).whenComplete((state, exception) -> {
+                        scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+                            CoordinatorContext ctx = contextOrThrow(tp);
+                            try {
+                                if (exception != null) throw exception;
+                                partitionWriter.registerListener(ctx.tp, highWatermarklistener);
+                                context.snapshotRegistry.getOrCreateSnapshot(0);
+                                context.coordinator.onLoaded();
+                                ctx.state = CoordinatorState.ACTIVE;
+                                log.info("Finished loading of metadata from {} with epoch {}.", tp, partitionEpoch);
+                            } catch (Throwable ex) {
+                                log.error("Failed to load metadata from {} due to {}.", tp, ex);
+                                ctx.state = CoordinatorState.FAILED;
+                            }
+                        });
+                    });
+                }
+            } else {
+                log.info("Ignoring loading metadata from {} since current epoch is {}.",

Review Comment:
   should we include the partitionEpoch?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class InternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        InternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            log.error("Execution of {} failed due to {}.", name, exception);
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                                The log context.
+     * @param processor                                 The event processor.
+     * @param partitionWriter                           The partition writer.
+     * @param loader                                    The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new InternalEvent(name, tp, op));
+    }
+
+    /**
+     * Builds a new coordinator state machine.
+     *
+     * @param context The coordinator context.
+     *
+     * @return The built coordinator state machine.
+     */
+    private S buildCoordinator(
+        CoordinatorContext context
+    ) {
+        return coordinatorBuilderSupplier
+            .get()
+            .withSnapshotRegistry(context.snapshotRegistry)
+            .build();
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.

Review Comment:
   can we give a bit more description on what the difference is between this and context.epoch?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class InternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        InternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            log.error("Execution of {} failed due to {}.", name, exception);
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                                The log context.
+     * @param processor                                 The event processor.
+     * @param partitionWriter                           The partition writer.
+     * @param loader                                    The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new InternalEvent(name, tp, op));
+    }
+
+    /**
+     * Builds a new coordinator state machine.
+     *
+     * @param context The coordinator context.
+     *
+     * @return The built coordinator state machine.
+     */
+    private S buildCoordinator(
+        CoordinatorContext context
+    ) {
+        return coordinatorBuilderSupplier
+            .get()
+            .withSnapshotRegistry(context.snapshotRegistry)
+            .build();
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+                if (context.state == CoordinatorState.LOADING) {

Review Comment:
   since we only run one event at a given time for a partition, my intuition tells me that we should not hit this condition. when would this be true?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218601535


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.

Review Comment:
   Or do we not deal with records at all because it is internal?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1214835504


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations

Review Comment:
   nit: exposes



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1220268287


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.

Review Comment:
   So in other words -- not writing or reading but doing some operation.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224000837


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

Review Comment:
   Right.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1221552609


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {

Review Comment:
   Yeah, I am not sure. In this case, complete is only useful in case of errors so I would not overuse it.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222148781


##########
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();

Review Comment:
   do we also want to check deregister was called? (I suppose we don't actually register, but the code path does contain this and the other test looks for registerListener)



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222029897


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.

Review Comment:
   I suppose not. It could also do without it. Up to you.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222169738


##########
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)
+        );

Review Comment:
   Should we check that the state is closed?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1217666174


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class InternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        InternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            log.error("Execution of {} failed due to {}.", name, exception);
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                                The log context.
+     * @param processor                                 The event processor.
+     * @param partitionWriter                           The partition writer.
+     * @param loader                                    The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new InternalEvent(name, tp, op));
+    }
+
+    /**
+     * Builds a new coordinator state machine.
+     *
+     * @param context The coordinator context.
+     *
+     * @return The built coordinator state machine.
+     */
+    private S buildCoordinator(
+        CoordinatorContext context
+    ) {
+        return coordinatorBuilderSupplier
+            .get()
+            .withSnapshotRegistry(context.snapshotRegistry)
+            .build();
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.

Review Comment:
   There are basically the same: `context.epoch = partitionEpoch`. Is the name confusing?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224025374


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new CoordinatorInternalEvent(name, tp, op));
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        // Touch the state to make the runtime immediately aware of the new coordinator.
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+
+            if (context.state == CoordinatorState.FAILED) {
+                // When the coordinator has failed, we create a new context instead of
+                // recycling the previous one because it is better to start from an
+                // empty state for timeline data structures.
+                coordinators.remove(tp);
+                context = getOrCreateContext(tp);
+            }
+
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+
+                switch (context.state) {
+                    case INITIAL:
+                        context.transitionTo(CoordinatorState.LOADING);
+                        loader.load(tp, context.coordinator).whenComplete((state, exception) -> {
+                            scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+                                CoordinatorContext ctx = contextOrThrow(tp);
+                                if (ctx.state != CoordinatorState.LOADING) {
+                                    log.info("Ignoring load completion from {} because context is in {} state.",
+                                        ctx.tp, ctx.state);
+                                    return;
+                                }
+                                try {
+                                    if (exception != null) throw exception;
+                                    ctx.transitionTo(CoordinatorState.ACTIVE);
+                                    log.info("Finished loading of metadata from {} with epoch {}.",
+                                        tp, partitionEpoch);
+                                } catch (Throwable ex) {
+                                    log.error("Failed to load metadata from {} with epoch {} due to {}.",
+                                        tp, partitionEpoch, ex.toString());
+                                    ctx.transitionTo(CoordinatorState.FAILED);
+                                }
+                            });
+                        });
+                        break;
+
+                    case LOADING:
+                        log.info("The coordinator {} is already loading metadata.", tp);
+                        break;
+
+                    case ACTIVE:
+                        log.info("The coordinator {} is already active.", tp);
+                        break;
+
+                    default:
+                        log.error("Cannot load coordinator {} in state {}.", tp, context.state);
+                }
+            } else {
+                log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.",
+                    context.tp, context.epoch, partitionEpoch);
+            }
+        });
+    }
+
+    /**
+     * Schedules the unloading of a coordinator. This is called when the broker is not the
+     * leader anymore.
+     *
+     * @param tp                The topic partition of the coordinator.
+     * @param partitionEpoch    The partition epoch.
+     */
+    public void scheduleUnloadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling unloading of metadata for {} with epoch {}", tp, partitionEpoch);
+
+        scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = contextOrThrow(tp);
+            if (context.epoch < partitionEpoch) {
+                log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+                context.transitionTo(CoordinatorState.CLOSED);
+                coordinators.remove(tp);
+                log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+            } else {
+                log.info("Ignored unloading metadata for {} in epoch {} since current epoch is {}.",
+                    tp, partitionEpoch, context.epoch);
+            }
+        });
+    }
+
+    /**
+     * Closes the runtime. This closes all the coordinators currently registered
+     * in the runtime.
+     *
+     * @throws Exception
+     */
+    public void close() throws Exception {
+        log.info("Closing coordinator runtime.");
+        // This close the processor, drain all the pending events and
+        // reject any new events.
+        processor.close();

Review Comment:
   I am not sure. I definitely use CloseQuietly when closing the runtime component but it seems better to raise the exception further if closing the processor fails here. That is because the rest of the closing procedure does not make sense if the closing the processor has failed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224010919


##########
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));

Review Comment:
   No, it won't. The last write here does not yield any records so a new snapshot is not created.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218533754


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;

Review Comment:
   so we can go from closed/failed back to loading?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218561662


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.

Review Comment:
   I see that this event will not return a response -- but does it also not write records? Is it read only? Or it it only on the in memory state?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1220560961


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.

Review Comment:
   Is this comment helpful 😅 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1223158227


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);

Review Comment:
   I am not aware of the threading model here but since we are using a concurrent hash map to store coordinators, I am assuming that this function can be accessed by multiple threads.
   
   In multi thread access is true for coordinator when this function is executing, then there is a bug here because there might be a context switch after we have retrieved the value using `get` and by the time we execute the if/else below, the value of this context might have changed. We should use atomic primitives with ConcurrentHashMaps such as `computeIfAbsent`



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);

Review Comment:
   can this be done async by background thread pool?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new CoordinatorInternalEvent(name, tp, op));
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        // Touch the state to make the runtime immediately aware of the new coordinator.
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+
+            if (context.state == CoordinatorState.FAILED) {
+                // When the coordinator has failed, we create a new context instead of
+                // recycling the previous one because it is better to start from an
+                // empty state for timeline data structures.
+                coordinators.remove(tp);
+                context = getOrCreateContext(tp);
+            }
+
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+
+                switch (context.state) {
+                    case INITIAL:
+                        context.transitionTo(CoordinatorState.LOADING);
+                        loader.load(tp, context.coordinator).whenComplete((state, exception) -> {
+                            scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+                                CoordinatorContext ctx = contextOrThrow(tp);
+                                if (ctx.state != CoordinatorState.LOADING) {
+                                    log.info("Ignoring load completion from {} because context is in {} state.",
+                                        ctx.tp, ctx.state);
+                                    return;
+                                }
+                                try {
+                                    if (exception != null) throw exception;
+                                    ctx.transitionTo(CoordinatorState.ACTIVE);
+                                    log.info("Finished loading of metadata from {} with epoch {}.",
+                                        tp, partitionEpoch);
+                                } catch (Throwable ex) {
+                                    log.error("Failed to load metadata from {} with epoch {} due to {}.",
+                                        tp, partitionEpoch, ex.toString());
+                                    ctx.transitionTo(CoordinatorState.FAILED);
+                                }
+                            });
+                        });
+                        break;
+
+                    case LOADING:
+                        log.info("The coordinator {} is already loading metadata.", tp);
+                        break;
+
+                    case ACTIVE:
+                        log.info("The coordinator {} is already active.", tp);
+                        break;
+
+                    default:
+                        log.error("Cannot load coordinator {} in state {}.", tp, context.state);
+                }
+            } else {
+                log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.",
+                    context.tp, context.epoch, partitionEpoch);
+            }
+        });
+    }
+
+    /**
+     * Schedules the unloading of a coordinator. This is called when the broker is not the
+     * leader anymore.
+     *
+     * @param tp                The topic partition of the coordinator.
+     * @param partitionEpoch    The partition epoch.
+     */
+    public void scheduleUnloadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling unloading of metadata for {} with epoch {}", tp, partitionEpoch);
+
+        scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = contextOrThrow(tp);
+            if (context.epoch < partitionEpoch) {
+                log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+                context.transitionTo(CoordinatorState.CLOSED);
+                coordinators.remove(tp);
+                log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+            } else {
+                log.info("Ignored unloading metadata for {} in epoch {} since current epoch is {}.",
+                    tp, partitionEpoch, context.epoch);
+            }
+        });
+    }
+
+    /**
+     * Closes the runtime. This closes all the coordinators currently registered
+     * in the runtime.
+     *
+     * @throws Exception
+     */
+    public void close() throws Exception {
+        log.info("Closing coordinator runtime.");
+        // This close the processor, drain all the pending events and
+        // reject any new events.
+        processor.close();

Review Comment:
   CloseQuietly?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);

Review Comment:
   you can get rid of the second `{}` because last parameter can be added as a exception without having to parameterize it. see: https://www.slf4j.org/faq.html#paramException



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new CoordinatorInternalEvent(name, tp, op));
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        // Touch the state to make the runtime immediately aware of the new coordinator.
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+
+            if (context.state == CoordinatorState.FAILED) {
+                // When the coordinator has failed, we create a new context instead of
+                // recycling the previous one because it is better to start from an
+                // empty state for timeline data structures.
+                coordinators.remove(tp);
+                context = getOrCreateContext(tp);
+            }
+
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+
+                switch (context.state) {
+                    case INITIAL:
+                        context.transitionTo(CoordinatorState.LOADING);
+                        loader.load(tp, context.coordinator).whenComplete((state, exception) -> {
+                            scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+                                CoordinatorContext ctx = contextOrThrow(tp);
+                                if (ctx.state != CoordinatorState.LOADING) {
+                                    log.info("Ignoring load completion from {} because context is in {} state.",
+                                        ctx.tp, ctx.state);
+                                    return;
+                                }
+                                try {
+                                    if (exception != null) throw exception;
+                                    ctx.transitionTo(CoordinatorState.ACTIVE);
+                                    log.info("Finished loading of metadata from {} with epoch {}.",
+                                        tp, partitionEpoch);
+                                } catch (Throwable ex) {
+                                    log.error("Failed to load metadata from {} with epoch {} due to {}.",
+                                        tp, partitionEpoch, ex.toString());
+                                    ctx.transitionTo(CoordinatorState.FAILED);
+                                }
+                            });
+                        });
+                        break;
+
+                    case LOADING:
+                        log.info("The coordinator {} is already loading metadata.", tp);
+                        break;
+
+                    case ACTIVE:
+                        log.info("The coordinator {} is already active.", tp);
+                        break;
+
+                    default:
+                        log.error("Cannot load coordinator {} in state {}.", tp, context.state);
+                }
+            } else {
+                log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.",
+                    context.tp, context.epoch, partitionEpoch);
+            }
+        });
+    }
+
+    /**
+     * Schedules the unloading of a coordinator. This is called when the broker is not the
+     * leader anymore.
+     *
+     * @param tp                The topic partition of the coordinator.
+     * @param partitionEpoch    The partition epoch.
+     */
+    public void scheduleUnloadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling unloading of metadata for {} with epoch {}", tp, partitionEpoch);
+
+        scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = contextOrThrow(tp);
+            if (context.epoch < partitionEpoch) {
+                log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+                context.transitionTo(CoordinatorState.CLOSED);
+                coordinators.remove(tp);
+                log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+            } else {
+                log.info("Ignored unloading metadata for {} in epoch {} since current epoch is {}.",
+                    tp, partitionEpoch, context.epoch);
+            }
+        });
+    }
+
+    /**
+     * Closes the runtime. This closes all the coordinators currently registered
+     * in the runtime.
+     *
+     * @throws Exception
+     */
+    public void close() throws Exception {
+        log.info("Closing coordinator runtime.");
+        // This close the processor, drain all the pending events and
+        // reject any new events.
+        processor.close();
+        // Unload all the coordinators.
+        coordinators.forEach((tp, context) -> {

Review Comment:
   note that this is not a thread safe operation since concurrentHashMap doesn't take an exclusive lock during forEach. Is that ok?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222175645


##########
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());

Review Comment:
   just for my understanding, we will always keep the latest snapshot in case we need to rollback?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224022949


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);

Review Comment:
   That's correct. `coordinators` is accessed by multiple threads. However, the runtime guarantees that the context for a given TopicPartition is never accessed concurrently - all the events of a TopicPartition are processed sequentially. This is why I don't have a lock for the context.
   
   Regarding your suggestion, I am not sure how you could use an atomic primitive of ConcurrentHashMaps to implement this logic. Could you elaborate a bit more on this?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224000539


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java:
##########
@@ -39,7 +40,7 @@ public class MultiThreadedEventProcessor implements CoordinatorEventProcessor {
     /**
      * The accumulator.
      */
-    private final EventAccumulator<Integer, CoordinatorEvent> accumulator;

Review Comment:
   Right. It was also just easier to use TopicPartition everywhere.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1217658963


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;

Review Comment:
   I think that I will use `coordinator` to stay consistent with the rest of the class.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1217663057


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.

Review Comment:
   Rephrases it.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222158858


##########
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);

Review Comment:
   Do we need to create a new coordinator here? We don't seem to verify that it is different.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224002293


##########
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:
   In this case, it fails because the context does not even exist.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1221551290


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.

Review Comment:
   Yeah, I agree... but it does not hurt, isn't it?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218999574


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;

Review Comment:
   very good question. it actually depends on whether we want to recycle the context or not. after thinking a bit more about it, i think that it is preferable to use reuse it (cause of the snapshot registry). i have updated the state machine and the code to reflect this.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);

Review Comment:
   almost. updated the code.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1219000861


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,

Review Comment:
   imagine a simple heartbeat request. we have to treat is as a write because it could alter the state but the heartbeat may not do anything.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1217668324


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received that last committed offset and it can use it to only
+         * access committed state in the timeline datastructures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class InternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        InternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            log.error("Execution of {} failed due to {}.", name, exception);
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                                The log context.
+     * @param processor                                 The event processor.
+     * @param partitionWriter                           The partition writer.
+     * @param loader                                    The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new InternalEvent(name, tp, op));
+    }
+
+    /**
+     * Builds a new coordinator state machine.
+     *
+     * @param context The coordinator context.
+     *
+     * @return The built coordinator state machine.
+     */
+    private S buildCoordinator(
+        CoordinatorContext context
+    ) {
+        return coordinatorBuilderSupplier
+            .get()
+            .withSnapshotRegistry(context.snapshotRegistry)
+            .build();
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+                if (context.state == CoordinatorState.LOADING) {

Review Comment:
   The loading is asynchronous so it could be that the leader epoch changes while we are already loading the coordinator.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1217659281


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Closes this coordinator. All the deferred events are failed
+         * with a NOT_COORDINATOR error.
+         */
+        private void close() {
+            log.debug("Closing coordinator for " + tp + ".");
+            state = CoordinatorState.CLOSED;
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            coordinator.onUnloaded();
+            log.debug("Coordinator for " + tp + " closed.");
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param state The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S state) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.

Review Comment:
   Correct.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218593915


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {

Review Comment:
   Complete is a bit strange here since we only call this in the error case. Would it make more sense to just call this in the finally block and if the error is null do nothing?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222117410


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

Review Comment:
   nit: we don't need to set 1 right? It will not be used.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222033908


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {

Review Comment:
   Yeah I guess if we don't use futures, it's not as important.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218537640


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);

Review Comment:
   are all of these methods safe to call from initial state? seems like that state transition is valid.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222202869


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

Review Comment:
   nit: the write is done (as we update lastWrittenOffset), but isDone signified that the write has been committed. Is this correct?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1217658117


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp

Review Comment:
   Correct. Every partition is mapped to a CondinatorContext and a Coordinator. The Coordinator here is basically the replicated state machine.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1214930728


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp

Review Comment:
   I was also wondering this, so thanks for bringing it up.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224015812


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

Review Comment:
   Yeah, this is correct. The future is only completed when the write is committed. Let me use committed in the comment.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1220554821


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {

Review Comment:
   The others followed a pattern where they both called complete, but maybe it's not a huge deal. Was suggesting a way to continue this pattern



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1214957871


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,959 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework expose an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL,
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING,
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE,
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED,
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {

Review Comment:
   nit: do we want a check that is the opposite of the updateLastWrittenOffset (ie, offset can not be greater than last written offset?)



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218561662


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.

Review Comment:
   I see that this event will not return a response -- but does it also not write records? Is it read only?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1219004026


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            log.error("Execution of {} failed due to {}.", name, exception);
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new CoordinatorInternalEvent(name, tp, op));
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+
+                switch (context.state) {
+                    case INITIAL:
+                    case FAILED:
+                    case CLOSED:
+                        context.transitionTo(CoordinatorState.LOADING);
+                        loader.load(tp, context.coordinator).whenComplete((state, exception) -> {
+                            scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+                                CoordinatorContext ctx = contextOrThrow(tp);
+                                try {
+                                    if (exception != null) throw exception;
+                                    ctx.transitionTo(CoordinatorState.ACTIVE);
+                                    log.info("Finished loading of metadata from {} with epoch {}.", tp, partitionEpoch);
+                                } catch (Throwable ex) {
+                                    log.error("Failed to load metadata from {} due to {}.", tp, ex);
+                                    ctx.transitionTo(CoordinatorState.FAILED);
+                                }
+                            });
+                        });
+                        break;
+
+                    case LOADING:
+                        log.info("The coordinator {} is already loading metadata.", tp);
+                        break;
+
+                    case ACTIVE:
+                        log.info("The coordinator {} is already active.", tp);
+                        break;
+                }
+            } else {
+                log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.",
+                    context.tp, context.epoch, partitionEpoch);
+            }
+        });
+    }
+
+    /**
+     * Schedules the unloading of a coordinator. This is called when the broker is not the
+     * leader anymore.
+     *
+     * @param tp                The topic partition of the coordinator.
+     * @param partitionEpoch    The partition epoch.
+     */
+    public void scheduleUnloadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling unloading of metadata for {} with epoch {}", tp, partitionEpoch);
+
+        scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = contextOrThrow(tp);
+            if (context.epoch < partitionEpoch) {

Review Comment:
   Right. The epoch is basically the leader epoch of the partition. The broker notifies us when a new ones comes and we update it here. However, we only load if we have to.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1219001617


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.

Review Comment:
   None of them. This is just a way to schedule an internal task (e.g. load/unload ops).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224571907


##########
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));

Review Comment:
   I see. The same happens on line 498 👍 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224014112


##########
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:
   The number of records does not matter here. The writer is configured to only accept one successful call to PartitionWriter#append. Let me extend the comment to make this clear.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224019689


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);

Review Comment:
   Hmm... All the events are already processed in a thread pool. We could defer those in another one but I don't really see the need for it for two reasons: 1) that would require to introduce locking because those are not thread safe; and 2) they should be quite fast operations.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222115234


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java:
##########
@@ -39,7 +40,7 @@ public class MultiThreadedEventProcessor implements CoordinatorEventProcessor {
     /**
      * The accumulator.
      */
-    private final EventAccumulator<Integer, CoordinatorEvent> accumulator;

Review Comment:
   Is this changed from integer to topic partition so that we can use different coordinator state partitions (ie consumer offsets vs transactional state)



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1222140560


##########
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:
   does this fail not because we aren't loaded, but because we haven't even started loading? On 201, it says we are still loading, but were able to get the context.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1224033285


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1040 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorBuilderSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator loading has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == LOADING;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    unload();
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    unload();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+
+        /**
+         * Unloads the coordinator.
+         */
+        private void unload() {
+            partitionWriter.deregisterListener(tp, highWatermarklistener);
+            deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+            if (coordinator != null) {
+                coordinator.onUnloaded();
+                coordinator = null;
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,
+                    // the response can be returned directly iff there are no pending write operations;
+                    // otherwise, the read needs to wait on the last write operation to be completed.
+                    OptionalLong pendingOffset = context.deferredEventQueue.highestPendingOffset();
+                    if (pendingOffset.isPresent()) {
+                        context.deferredEventQueue.add(pendingOffset.getAsLong(), this);
+                    } else {
+                        complete(null);
+                    }
+                } else {
+                    // If the records are not empty, first, they are applied to the state machine,
+                    // second, then are written to the partition/log, and finally, the response
+                    // is put into the deferred event queue.
+                    try {
+                        // Apply the records to the state machine.
+                        result.records().forEach(context.coordinator::replay);
+
+                        // Write the records to the log and update the last written
+                        // offset.
+                        long offset = partitionWriter.append(tp, result.records());
+                        context.updateLastWrittenOffset(offset);
+
+                        // Add the response to the deferred queue.
+                        if (!future.isDone()) {
+                            context.deferredEventQueue.add(offset, this);
+                        } else {
+                            complete(null);
+                        }
+                    } catch (Throwable t) {
+                        context.revertLastWrittenOffset(prevLastWrittenOffset);
+                        complete(t);
+                    }
+                }
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the write operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(result.response());
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorWriteEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator read operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     */
+    public interface CoordinatorReadOperation<S, T> {
+        /**
+         * Generates the response to implement this coordinator read operation. A read
+         * operation received the last committed offset. It must use it to ensure that
+         * it does not read uncommitted data from the timeline data structures.
+         *
+         * @param state     The coordinator state machine.
+         * @param offset    The last committed offset.
+         * @return A response.
+         * @throws KafkaException
+         */
+        T generateResponse(S state, long offset) throws KafkaException;
+    }
+
+    /**
+     * A coordinator that reads the committed coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorReadEvent<T> implements CoordinatorEvent {
+        /**
+         * The topic partition that this read event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The read operation to execute.
+         */
+        final CoordinatorReadOperation<S, T> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the read operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the read operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        T response;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The read operation.
+         */
+        CoordinatorReadEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorReadOperation<S, T> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                // Execute the read operation.
+                response = op.generateResponse(
+                    context.coordinator,
+                    context.lastCommittedOffset
+                );
+
+                // The response can be completed immediately.
+                complete(null);
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Completes the future with either the result of the read operation
+         * or the provided exception.
+         *
+         * @param exception The exception to complete the future with.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                future.complete(response);
+            } else {
+                future.completeExceptionally(exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "CoordinatorReadEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * A coordinator internal event.
+     */
+    class CoordinatorInternalEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final Runnable op;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        CoordinatorInternalEvent(
+            String name,
+            TopicPartition tp,
+            Runnable op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                op.run();
+            } catch (Throwable t) {
+                complete(t);
+            }
+        }
+
+        /**
+         * Logs any exceptions thrown while the event is executed.
+         *
+         * @param exception The exception.
+         */
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                log.error("Execution of {} failed due to {}.", name, exception);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "InternalEvent(name=" + name + ")";
+        }
+    }
+
+    /**
+     * Partition listener to be notified when the high watermark of the partitions
+     * backing the coordinator are updated.
+     */
+    class HighWatermarkListener implements PartitionWriter.Listener {
+        /**
+         * Updates the high watermark of the corresponding coordinator.
+         *
+         * @param tp        The topic partition.
+         * @param offset    The new high watermark.
+         */
+        @Override
+        public void onHighWatermarkUpdated(
+            TopicPartition tp,
+            long offset
+        ) {
+            log.debug("High watermark of {} incremented to {}.", tp, offset);
+            scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> {
+                contextOrThrow(tp).updateLastCommittedOffset(offset);
+            });
+        }
+    }
+
+    /**
+     * The log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The logger.
+     */
+    private final Logger log;
+
+    /**
+     * The coordinators keyed by topic partition.
+     */
+    private final ConcurrentHashMap<TopicPartition, CoordinatorContext> coordinators;
+
+    /**
+     * The event processor used by the runtime.
+     */
+    private final CoordinatorEventProcessor processor;
+
+    /**
+     * The partition writer used by the runtime to persist records.
+     */
+    private final PartitionWriter<U> partitionWriter;
+
+    /**
+     * The high watermark listener registered to all the partitions
+     * backing the coordinators.
+     */
+    private final HighWatermarkListener highWatermarklistener;
+
+    /**
+     * The coordinator loaded used by the runtime.
+     */
+    private final CoordinatorLoader<U> loader;
+
+    /**
+     * The coordinator state machine builder used by the runtime
+     * to instantiate a coordinator.
+     */
+    private final CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+    /**
+     * Constructor.
+     *
+     * @param logContext                    The log context.
+     * @param processor                     The event processor.
+     * @param partitionWriter               The partition writer.
+     * @param loader                        The coordinator loader.
+     * @param coordinatorBuilderSupplier    The coordinator builder.
+     */
+    private CoordinatorRuntime(
+        LogContext logContext,
+        CoordinatorEventProcessor processor,
+        PartitionWriter<U> partitionWriter,
+        CoordinatorLoader<U> loader,
+        CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier
+    ) {
+        this.logContext = logContext;
+        this.log = logContext.logger(CoordinatorRuntime.class);
+        this.coordinators = new ConcurrentHashMap<>();
+        this.processor = processor;
+        this.partitionWriter = partitionWriter;
+        this.highWatermarklistener = new HighWatermarkListener();
+        this.loader = loader;
+        this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+    }
+
+    /**
+     * @return The coordinator context if the coordinator is active or an exception otherwise.
+     * @throws NotCoordinatorException
+     * @throws CoordinatorLoadInProgressException
+     */
+    private CoordinatorContext activeContextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            switch (context.state) {
+                case INITIAL:
+                case FAILED:
+                case CLOSED:
+                    throw Errors.NOT_COORDINATOR.exception();
+
+                case LOADING:
+                    throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception();
+            }
+        }
+
+        return context;
+    }
+
+    /**
+     * @return The coordinator context. It is created if it does not exist.
+     */
+    private CoordinatorContext getOrCreateContext(TopicPartition tp) {
+        return coordinators.computeIfAbsent(tp, CoordinatorContext::new);
+    }
+
+    /**
+     * @return The coordinator context or thrown an exception if it does
+     * not exist.
+     * @throws NotCoordinatorException
+     * Package private for testing.
+     */
+    CoordinatorContext contextOrThrow(TopicPartition tp) {
+        CoordinatorContext context = coordinators.get(tp);
+
+        if (context == null) {
+            throw Errors.NOT_COORDINATOR.exception();
+        } else {
+            return context;
+        }
+    }
+
+    /**
+     * Schedules a write operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The write operation.
+     *
+     * @return A future that will be completed with the result of the write operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleWriteOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorWriteOperation<S, T, U> op
+    ) {
+        log.debug("Scheduled execution of write operation {}.", name);
+        CoordinatorWriteEvent<T> event = new CoordinatorWriteEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules a read operation.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The read operation.
+     *
+     * @return A future that will be completed with the result of the read operation
+     * when the operation is completed or an exception if the write operation failed.
+     *
+     * @param <T> The type of the result.
+     */
+    public <T> CompletableFuture<T> scheduleReadOperation(
+        String name,
+        TopicPartition tp,
+        CoordinatorReadOperation<S, T> op
+    ) {
+        log.debug("Scheduled execution of read operation {}.", name);
+        CoordinatorReadEvent<T> event = new CoordinatorReadEvent<>(name, tp, op);
+        processor.enqueue(event);
+        return event.future;
+    }
+
+    /**
+     * Schedules an internal event.
+     *
+     * @param name  The name of the write operation.
+     * @param tp    The address of the coordinator (aka its topic-partitions).
+     * @param op    The operation.
+     */
+    private void scheduleInternalOperation(
+        String name,
+        TopicPartition tp,
+        Runnable op
+    ) {
+        log.debug("Scheduled execution of internal operation {}.", name);
+        processor.enqueue(new CoordinatorInternalEvent(name, tp, op));
+    }
+
+    /**
+     * @return The topic partitions of the coordinators currently registered in the
+     * runtime.
+     */
+    public Set<TopicPartition> partitions() {
+        return new HashSet<>(coordinators.keySet());
+    }
+
+    /**
+     * Schedules the loading of a coordinator. This is called when the broker is elected as
+     * the leader for a partition.
+     *
+     * @param tp                The topic partition of the coordinator. Records from this
+     *                          partitions will be read and applied to the coordinator.
+     * @param partitionEpoch    The epoch of the partition.
+     */
+    public void scheduleLoadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling loading of metadata from {} with epoch {}", tp, partitionEpoch);
+        // Touch the state to make the runtime immediately aware of the new coordinator.
+        getOrCreateContext(tp);
+
+        scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = getOrCreateContext(tp);
+
+            if (context.state == CoordinatorState.FAILED) {
+                // When the coordinator has failed, we create a new context instead of
+                // recycling the previous one because it is better to start from an
+                // empty state for timeline data structures.
+                coordinators.remove(tp);
+                context = getOrCreateContext(tp);
+            }
+
+            if (context.epoch < partitionEpoch) {
+                context.epoch = partitionEpoch;
+
+                switch (context.state) {
+                    case INITIAL:
+                        context.transitionTo(CoordinatorState.LOADING);
+                        loader.load(tp, context.coordinator).whenComplete((state, exception) -> {
+                            scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+                                CoordinatorContext ctx = contextOrThrow(tp);
+                                if (ctx.state != CoordinatorState.LOADING) {
+                                    log.info("Ignoring load completion from {} because context is in {} state.",
+                                        ctx.tp, ctx.state);
+                                    return;
+                                }
+                                try {
+                                    if (exception != null) throw exception;
+                                    ctx.transitionTo(CoordinatorState.ACTIVE);
+                                    log.info("Finished loading of metadata from {} with epoch {}.",
+                                        tp, partitionEpoch);
+                                } catch (Throwable ex) {
+                                    log.error("Failed to load metadata from {} with epoch {} due to {}.",
+                                        tp, partitionEpoch, ex.toString());
+                                    ctx.transitionTo(CoordinatorState.FAILED);
+                                }
+                            });
+                        });
+                        break;
+
+                    case LOADING:
+                        log.info("The coordinator {} is already loading metadata.", tp);
+                        break;
+
+                    case ACTIVE:
+                        log.info("The coordinator {} is already active.", tp);
+                        break;
+
+                    default:
+                        log.error("Cannot load coordinator {} in state {}.", tp, context.state);
+                }
+            } else {
+                log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.",
+                    context.tp, context.epoch, partitionEpoch);
+            }
+        });
+    }
+
+    /**
+     * Schedules the unloading of a coordinator. This is called when the broker is not the
+     * leader anymore.
+     *
+     * @param tp                The topic partition of the coordinator.
+     * @param partitionEpoch    The partition epoch.
+     */
+    public void scheduleUnloadOperation(
+        TopicPartition tp,
+        int partitionEpoch
+    ) {
+        log.info("Scheduling unloading of metadata for {} with epoch {}", tp, partitionEpoch);
+
+        scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> {
+            CoordinatorContext context = contextOrThrow(tp);
+            if (context.epoch < partitionEpoch) {
+                log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+                context.transitionTo(CoordinatorState.CLOSED);
+                coordinators.remove(tp);
+                log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch);
+            } else {
+                log.info("Ignored unloading metadata for {} in epoch {} since current epoch is {}.",
+                    tp, partitionEpoch, context.epoch);
+            }
+        });
+    }
+
+    /**
+     * Closes the runtime. This closes all the coordinators currently registered
+     * in the runtime.
+     *
+     * @throws Exception
+     */
+    public void close() throws Exception {
+        log.info("Closing coordinator runtime.");
+        // This close the processor, drain all the pending events and
+        // reject any new events.
+        processor.close();
+        // Unload all the coordinators.
+        coordinators.forEach((tp, context) -> {

Review Comment:
   This should be OK. Closing the processor guarantees that `coordinators` is not accessed anymore. Your comment made me think that I should add an atomic boolean to make the close method idempotent.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1226541305


##########
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:
   Yeah, I agree that the records are not great in this test. For the context, the runtime always write what it gets. It does not compare the records themselves.
   
   I have updated the writer to fail if the number of records in a single write is greater than a threshold. I think that it will be less confusing this way. What do you think?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac merged pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac merged PR #13795:
URL: https://github.com/apache/kafka/pull/13795


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13795: KAFKA-14462; [17/N] Add CoordinatorRuntime

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13795:
URL: https://github.com/apache/kafka/pull/13795#discussion_r1218547896


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.NotCoordinatorException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.deferred.DeferredEvent;
+import org.apache.kafka.deferred.DeferredEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.HashSet;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator
+ * or the transaction coordinator.
+ *
+ * The runtime framework maps each underlying partitions (e.g. __consumer_offsets) that that broker is a
+ * leader of to a coordinator replicated state machine. A replicated state machine holds the hard and soft
+ * state of all the objects (e.g. groups or offsets) assigned to the partition. The hard state is stored in
+ * timeline datastructures backed by a SnapshotRegistry. The runtime supports two type of operations
+ * on state machines: (1) Writes and (2) Reads.
+ *
+ * (1) A write operation, aka a request, can read the full and potentially **uncommitted** state from state
+ * machine to handle the operation. A write operation typically generates a response and a list of
+ * records. The records are applies to the state machine and persisted to the partition. The response
+ * is parked until the records are committed and delivered when they are.
+ *
+ * (2) A read operation, aka a request, can only read the committed state from the state machine to handle
+ * the operation. A read operation typically generates a response that is immediately completed.
+ *
+ * The runtime framework exposes an asynchronous, future based, API to the world. All the operations
+ * are executed by an CoordinatorEventProcessor. The processor guarantees that operations for a
+ * single partition or state machine are not processed concurrently.
+ *
+ * @param <S> The type of the state machine.
+ * @param <U> The type of the record.
+ */
+public class CoordinatorRuntime<S extends Coordinator<U>, U> {
+
+    /**
+     * Builder to create a CoordinatorRuntime.
+     *
+     * @param <S> The type of the state machine.
+     * @param <U> The type of the record.
+     */
+    public static class Builder<S extends Coordinator<U>, U> {
+        private LogContext logContext;
+        private CoordinatorEventProcessor eventProcessor;
+        private PartitionWriter<U> partitionWriter;
+        private CoordinatorLoader<U> loader;
+        private CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier;
+
+        public Builder<S, U> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder<S, U> withEventProcessor(CoordinatorEventProcessor eventProcessor) {
+            this.eventProcessor = eventProcessor;
+            return this;
+        }
+
+        public Builder<S, U> withPartitionWriter(PartitionWriter<U> partitionWriter) {
+            this.partitionWriter = partitionWriter;
+            return this;
+        }
+
+        public Builder<S, U> withLoader(CoordinatorLoader<U> loader) {
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder<S, U> withCoordinatorStateMachineSupplier(CoordinatorBuilderSupplier<S, U> coordinatorBuilderSupplier) {
+            this.coordinatorBuilderSupplier = coordinatorBuilderSupplier;
+            return this;
+        }
+
+        public CoordinatorRuntime<S, U> build() {
+            if (logContext == null)
+                logContext = new LogContext();
+            if (eventProcessor == null)
+                throw new IllegalArgumentException("Event processor must be set.");
+            if (partitionWriter == null)
+                throw new IllegalArgumentException("Partition write must be set.");
+            if (loader == null)
+                throw new IllegalArgumentException("Loader must be set.");
+            if (coordinatorBuilderSupplier == null)
+                throw new IllegalArgumentException("State machine supplier must be set.");
+
+            return new CoordinatorRuntime<>(
+                logContext,
+                eventProcessor,
+                partitionWriter,
+                loader,
+                coordinatorBuilderSupplier
+            );
+        }
+    }
+
+    /**
+     * The various state that a coordinator for a partition can be in.
+     */
+    enum CoordinatorState {
+        /**
+         * Initial state when a coordinator is created.
+         */
+        INITIAL {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return false;
+            }
+        },
+
+        /**
+         * The coordinator is being loaded.
+         */
+        LOADING {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == INITIAL || state == CLOSED || state == FAILED;
+            }
+        },
+
+        /**
+         * The coordinator is active and can service requests.
+         */
+        ACTIVE {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return state == ACTIVE || state == LOADING;
+            }
+        },
+
+        /**
+         * The coordinator is closed.
+         */
+        CLOSED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        },
+
+        /**
+         * The coordinator has failed.
+         */
+        FAILED {
+            @Override
+            boolean canTransitionFrom(CoordinatorState state) {
+                return true;
+            }
+        };
+
+        abstract boolean canTransitionFrom(CoordinatorState state);
+    }
+
+    /**
+     * CoordinatorContext holds all the metadata around a coordinator state machine.
+     */
+    class CoordinatorContext {
+        /**
+         * The topic partition backing the coordinator.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The snapshot registry backing the coordinator.
+         */
+        final SnapshotRegistry snapshotRegistry;
+
+        /**
+         * The deferred event queue used to park events waiting
+         * on records to be committed.
+         */
+        final DeferredEventQueue deferredEventQueue;
+
+        /**
+         * The current state.
+         */
+        volatile CoordinatorState state;
+
+        /**
+         * The actual state machine.
+         */
+        volatile S coordinator;
+
+        /**
+         * The current epoch of the coordinator. This represents
+         * the epoch of the partition leader.
+         */
+        volatile int epoch;
+
+        /**
+         * The last offset written to the partition.
+         */
+        volatile long lastWrittenOffset;
+
+        /**
+         * The last offset committed. This represents the high
+         * watermark of the partition.
+         */
+        volatile long lastCommittedOffset;
+
+        /**
+         * Constructor.
+         *
+         * @param tp The topic partition of the coordinator.
+         */
+        private CoordinatorContext(
+            TopicPartition tp
+        ) {
+            this.tp = tp;
+            this.snapshotRegistry = new SnapshotRegistry(logContext);
+            this.deferredEventQueue = new DeferredEventQueue(logContext);
+            this.state = CoordinatorState.INITIAL;
+            this.epoch = -1;
+            this.lastWrittenOffset = 0L;
+            this.lastCommittedOffset = 0L;
+        }
+
+        /**
+         * Updates the last written offset. This also create a new snapshot
+         * in the snapshot registry.
+         *
+         * @param offset The new last written offset.
+         */
+        private void updateLastWrittenOffset(
+            long offset
+        ) {
+            if (offset <= lastWrittenOffset) {
+                throw new IllegalStateException("New last written offset " + offset + " of " + tp +
+                    " must be larger than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Update last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.getOrCreateSnapshot(offset);
+        }
+
+        /**
+         * Reverts the last written offset. This also reverts the snapshot
+         * registry to this offset. All the changes applied after the offset
+         * are lost.
+         *
+         * @param offset The offset to revert to.
+         */
+        private void revertLastWrittenOffset(
+            long offset
+        ) {
+            if (offset > lastWrittenOffset) {
+                throw new IllegalStateException("New offset " + offset + " of " + tp +
+                    " must be smaller than " + lastWrittenOffset + ".");
+            }
+
+            log.debug("Revert last written offset of {} to {}.", tp, offset);
+            lastWrittenOffset = offset;
+            snapshotRegistry.revertToSnapshot(offset);
+        }
+
+        /**
+         * Updates the last committed offset. This completes all the deferred
+         * events waiting on this offset. This also cleanups all the snapshots
+         * prior to this offset.
+         *
+         * @param offset The new last committed offset.
+         */
+        private void updateLastCommittedOffset(
+            long offset
+        ) {
+            if (offset <= lastCommittedOffset) {
+                throw new IllegalStateException("New committed offset " + offset + " of " + tp +
+                    " must be larger than " + lastCommittedOffset + ".");
+            }
+
+            log.debug("Update committed offset of {} to {}.", tp, offset);
+            lastCommittedOffset = offset;
+            deferredEventQueue.completeUpTo(offset);
+            snapshotRegistry.deleteSnapshotsUpTo(offset);
+        }
+
+        /**
+         * Transitions to the new state.
+         *
+         * @param newState The new state.
+         */
+        private void transitionTo(
+            CoordinatorState newState
+        ) {
+            if (!newState.canTransitionFrom(state)) {
+                throw new IllegalStateException("Cannot transition from " + state + " to " + newState);
+            }
+
+            log.debug("Transition from {} to {}.", state, newState);
+            switch (newState) {
+                case LOADING:
+                    state = CoordinatorState.LOADING;
+                    coordinator = coordinatorBuilderSupplier
+                        .get()
+                        .withSnapshotRegistry(snapshotRegistry)
+                        .build();
+                    break;
+
+                case ACTIVE:
+                    state = CoordinatorState.ACTIVE;
+                    partitionWriter.registerListener(tp, highWatermarklistener);
+                    snapshotRegistry.getOrCreateSnapshot(0);
+                    coordinator.onLoaded();
+                    break;
+
+                case FAILED:
+                    state = CoordinatorState.FAILED;
+                    break;
+
+                case CLOSED:
+                    state = CoordinatorState.CLOSED;
+                    partitionWriter.deregisterListener(tp, highWatermarklistener);
+                    deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception());
+                    coordinator.onUnloaded();
+                    break;
+
+                default:
+                    throw new IllegalArgumentException("Transitioning to " + newState + " is not supported.");
+            }
+        }
+    }
+
+    /**
+     * A coordinator write operation.
+     *
+     * @param <S> The type of the coordinator state machine.
+     * @param <T> The type of the response.
+     * @param <U> The type of the records.
+     */
+    public interface CoordinatorWriteOperation<S, T, U> {
+        /**
+         * Generates the records needed to implement this coordinator write operation. In general,
+         * this operation should not modify the hard state of the coordinator. That modifications
+         * will happen later on, when the records generated by this function are applied to the
+         * coordinator.
+         *
+         * @param coordinator The coordinator state machine.
+         * @return A result containing a list of records and the RPC result.
+         * @throws KafkaException
+         */
+        CoordinatorResult<T, U> generateRecordsAndResult(S coordinator) throws KafkaException;
+    }
+
+    /**
+     * A coordinator event that modifies the coordinator state.
+     *
+     * @param <T> The type of the response.
+     */
+    class CoordinatorWriteEvent<T> implements CoordinatorEvent, DeferredEvent {
+        /**
+         * The topic partition that this write event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The write operation to execute.
+         */
+        final CoordinatorWriteOperation<S, T, U> op;
+
+        /**
+         * The future that will be completed with the response
+         * generated by the write operation or an error.
+         */
+        final CompletableFuture<T> future;
+
+        /**
+         * The result of the write operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<T, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The write operation.
+         */
+        CoordinatorWriteEvent(
+            String name,
+            TopicPartition tp,
+            CoordinatorWriteOperation<S, T, U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+            this.future = new CompletableFuture<>();
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+                long prevLastWrittenOffset = context.lastWrittenOffset;
+
+                // Execute the operation.
+                result = op.generateRecordsAndResult(context.coordinator);
+
+                if (result.records().isEmpty()) {
+                    // If the records are empty, it was a read operation after all. In this case,

Review Comment:
   What scenarios would we have a write event that was a "read operation after all"?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org