smjn commented on code in PR #17270: URL: https://github.com/apache/kafka/pull/17270#discussion_r1817288320
########## share/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java: ########## @@ -0,0 +1,315 @@ +/* + * 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.server.share.persister; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ReadShareGroupStateResponse; +import org.apache.kafka.common.requests.WriteShareGroupStateResponse; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +/** + * The default implementation of the {@link Persister} interface which is used by the + * group coordinator and share-partition leaders to manage the durable share-partition state. + * This implementation uses inter-broker RPCs to make requests to the share coordinator + * which is responsible for persisting the share-partition state. + */ +public class DefaultStatePersister implements Persister { + private final PersisterStateManager stateManager; + + private static final Logger log = LoggerFactory.getLogger(DefaultStatePersister.class); + + public DefaultStatePersister(PersisterStateManager stateManager) { + this.stateManager = stateManager; + this.stateManager.start(); + } + + @Override + public void stop() { + try { + if (stateManager != null) { + stateManager.stop(); + } + } catch (Exception e) { + log.error("Unable to stop state manager", e); + } + } + + /** + * Used by the group coordinator to initialize the share-partition state. + * This is an inter-broker RPC authorized as a cluster action. + * + * @param request InitializeShareGroupStateParameters + * @return A completable future of InitializeShareGroupStateResult + */ + public CompletableFuture<InitializeShareGroupStateResult> initializeState(InitializeShareGroupStateParameters request) throws IllegalArgumentException { + throw new RuntimeException("not implemented"); + } + + /** + * Used by share-partition leaders to write share-partition state to a share coordinator. + * This is an inter-broker RPC authorized as a cluster action. + * + * @param request WriteShareGroupStateParameters + * @return A completable future of WriteShareGroupStateResult + */ + public CompletableFuture<WriteShareGroupStateResult> writeState(WriteShareGroupStateParameters request) throws IllegalArgumentException { + validate(request); + GroupTopicPartitionData<PartitionStateBatchData> gtp = request.groupTopicPartitionData(); + String groupId = gtp.groupId(); + + Map<Uuid, Map<Integer, CompletableFuture<WriteShareGroupStateResponse>>> futureMap = new HashMap<>(); + List<PersisterStateManager.WriteStateHandler> handlers = new ArrayList<>(); + + gtp.topicsData().forEach(topicData -> { + topicData.partitions().forEach(partitionData -> { + CompletableFuture<WriteShareGroupStateResponse> future = futureMap + .computeIfAbsent(topicData.topicId(), k -> new HashMap<>()) + .computeIfAbsent(partitionData.partition(), k -> new CompletableFuture<>()); + + handlers.add( + stateManager.new WriteStateHandler( + groupId, + topicData.topicId(), + partitionData.partition(), + partitionData.stateEpoch(), + partitionData.leaderEpoch(), + partitionData.startOffset(), + partitionData.stateBatches(), + future, null) + ); + }); + }); + + for (PersisterStateManager.PersisterStateManagerHandler handler : handlers) { + stateManager.enqueue(handler); + } + + CompletableFuture<Void> combinedFuture = CompletableFuture.allOf( + handlers.stream() + .map(PersisterStateManager.WriteStateHandler::result) + .toArray(CompletableFuture[]::new)); + + return combinedFuture.thenApply(v -> { + List<TopicData<PartitionErrorData>> topicsData = futureMap.keySet().stream() + .map(topicId -> { + List<PartitionErrorData> partitionErrData = futureMap.get(topicId).entrySet().stream() + .map(partitionFuture -> { + int partition = partitionFuture.getKey(); + CompletableFuture<WriteShareGroupStateResponse> future = partitionFuture.getValue(); + try { + WriteShareGroupStateResponse partitionResponse = future.get(); + return partitionResponse.data().results().get(0).partitions().stream() Review Comment: So in the default state persister read/write call, the sender can send a request which can have: ``` groupId, [ topic1:[part1, part2], topic2:[part1], topic3:[part5] . . . ] ``` However, we are not sending these requests as is - we explode them, into multiple requests: r1: groupId, topic1, part1 r2: groupId, topic1, part2 r3: groupId, topic2, part1 ... each request is then encapsulated in a handler and then enqueued in the PersisterStateManager. Exploding them is necessary since the share coordinator is keyed on a single (groupId, topic1, part1) so the entire request as sent by the caller is not serviceable by a single coordinator node. THIS IS THE REASON, we care about only the first entry. One we get the results back from the handlers (futures complete). We combine the fragments back to a response appropriate for the original large request. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
