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-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org