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

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

Reply via email to