smjn commented on code in PR #17011:
URL: https://github.com/apache/kafka/pull/17011#discussion_r1749650268


##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java:
##########
@@ -17,58 +17,552 @@
 
 package org.apache.kafka.coordinator.share;
 
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.message.ReadShareGroupStateRequestData;
 import org.apache.kafka.common.message.ReadShareGroupStateResponseData;
 import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
 import org.apache.kafka.common.message.WriteShareGroupStateResponseData;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
 import org.apache.kafka.common.requests.RequestContext;
 import org.apache.kafka.common.requests.TransactionResult;
+import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
 import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
 import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
 import org.apache.kafka.coordinator.common.runtime.CoordinatorShard;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorShardBuilder;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorTimer;
+import org.apache.kafka.coordinator.share.generated.ShareSnapshotKey;
+import org.apache.kafka.coordinator.share.generated.ShareSnapshotValue;
+import org.apache.kafka.coordinator.share.generated.ShareUpdateKey;
+import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
+import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
+import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetricsShard;
 import org.apache.kafka.image.MetadataDelta;
 import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.server.config.ShareCoordinatorConfig;
+import org.apache.kafka.server.group.share.PartitionFactory;
+import org.apache.kafka.server.group.share.SharePartitionKey;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
 
-/**
- * The share coordinator shard is a replicated state machine that manages the 
metadata of all
- * share partitions. It holds the hard and the soft state of the share 
partitions. This class
- * has two kinds of methods:
- * 1) The request handlers which handle the requests and generate a response 
and records to
- *    mutate the hard state. Those records will be written by the runtime and 
applied to the
- *    hard state via the replay methods.
- * 2) The replay methods which apply records to the hard state. Those are used 
in the request
- *    handling as well as during the initial loading of the records from the 
partitions.
- */
 public class ShareCoordinatorShard implements 
CoordinatorShard<CoordinatorRecord> {
+    private final Logger log;
+    private final Time time;
+    private final CoordinatorTimer<Void, CoordinatorRecord> timer;
+    private final ShareCoordinatorConfig config;
+    private final CoordinatorMetrics coordinatorMetrics;
+    private final CoordinatorMetricsShard metricsShard;
+    private final TimelineHashMap<SharePartitionKey, ShareGroupOffset> 
shareStateMap;  // coord key -> ShareGroupOffset
+    private final TimelineHashMap<SharePartitionKey, Integer> leaderEpochMap;
+    private final TimelineHashMap<SharePartitionKey, Integer> 
snapshotUpdateCount;
+    private final TimelineHashMap<SharePartitionKey, Integer> stateEpochMap;
+    private MetadataImage metadataImage;
+    private final int snapshotUpdateRecordsPerSnapshot;
+
+    public static final Exception NULL_TOPIC_ID = new Exception("The topic id 
cannot be null.");
+    public static final Exception NEGATIVE_PARTITION_ID = new Exception("The 
partition id cannot be a negative number.");
+
+    public static class Builder implements 
CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> {
+        private ShareCoordinatorConfig config;
+        private LogContext logContext;
+        private SnapshotRegistry snapshotRegistry;
+        private Time time;
+        private CoordinatorTimer<Void, CoordinatorRecord> timer;
+        private CoordinatorMetrics coordinatorMetrics;
+        private TopicPartition topicPartition;
+
+        public Builder(ShareCoordinatorConfig config) {
+            this.config = config;
+        }
+
+        @Override
+        public CoordinatorShardBuilder<ShareCoordinatorShard, 
CoordinatorRecord> withSnapshotRegistry(SnapshotRegistry snapshotRegistry) {
+            this.snapshotRegistry = snapshotRegistry;
+            return this;
+        }
+
+        @Override
+        public CoordinatorShardBuilder<ShareCoordinatorShard, 
CoordinatorRecord> withLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        @Override
+        public CoordinatorShardBuilder<ShareCoordinatorShard, 
CoordinatorRecord> withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        @Override
+        public CoordinatorShardBuilder<ShareCoordinatorShard, 
CoordinatorRecord> withTimer(CoordinatorTimer<Void, CoordinatorRecord> timer) {
+            this.timer = timer;
+            return this;
+        }
+
+        @Override
+        public CoordinatorShardBuilder<ShareCoordinatorShard, 
CoordinatorRecord> withCoordinatorMetrics(CoordinatorMetrics 
coordinatorMetrics) {
+            this.coordinatorMetrics = coordinatorMetrics;
+            return this;
+        }
+
+        @Override
+        public CoordinatorShardBuilder<ShareCoordinatorShard, 
CoordinatorRecord> withTopicPartition(TopicPartition topicPartition) {
+            this.topicPartition = topicPartition;
+            return this;
+        }
+
+        @Override
+        @SuppressWarnings("NPathComplexity")
+        public ShareCoordinatorShard build() {
+            if (logContext == null) logContext = new LogContext();
+            if (config == null)
+                throw new IllegalArgumentException("Config must be set.");
+            if (snapshotRegistry == null)
+                throw new IllegalArgumentException("SnapshotRegistry must be 
set.");
+            if (time == null)
+                throw new IllegalArgumentException("Time must be set.");
+            if (timer == null)
+                throw new IllegalArgumentException("Timer must be set.");
+            if (coordinatorMetrics == null || !(coordinatorMetrics instanceof 
ShareCoordinatorMetrics))
+                throw new IllegalArgumentException("CoordinatorMetrics must be 
set and be of type ShareCoordinatorMetrics.");
+            if (topicPartition == null)
+                throw new IllegalArgumentException("TopicPartition must be 
set.");
+
+            ShareCoordinatorMetricsShard metricsShard = 
((ShareCoordinatorMetrics) coordinatorMetrics)
+                .newMetricsShard(snapshotRegistry, topicPartition);
+
+            return new ShareCoordinatorShard(
+                logContext,
+                time,
+                timer,
+                config,
+                coordinatorMetrics,
+                metricsShard,
+                snapshotRegistry
+            );
+        }
+    }
+
+    ShareCoordinatorShard(
+        LogContext logContext,
+        Time time,
+        CoordinatorTimer<Void, CoordinatorRecord> timer,
+        ShareCoordinatorConfig config,
+        CoordinatorMetrics coordinatorMetrics,
+        CoordinatorMetricsShard metricsShard,
+        SnapshotRegistry snapshotRegistry
+    ) {
+        this.log = logContext.logger(ShareCoordinatorShard.class);
+        this.time = time;
+        this.timer = timer;
+        this.config = config;
+        this.coordinatorMetrics = coordinatorMetrics;
+        this.metricsShard = metricsShard;
+        this.shareStateMap = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.leaderEpochMap = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.snapshotUpdateCount = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.stateEpochMap = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.snapshotUpdateRecordsPerSnapshot = 
config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot();
+    }
+
     @Override
     public void onLoaded(MetadataImage newImage) {
-        CoordinatorShard.super.onLoaded(newImage);
+        coordinatorMetrics.activateMetricsShard(metricsShard);
     }
 
     @Override
     public void onNewMetadataImage(MetadataImage newImage, MetadataDelta 
delta) {
-        CoordinatorShard.super.onNewMetadataImage(newImage, delta);
+        this.metadataImage = newImage;
     }
 
     @Override
     public void onUnloaded() {
-        CoordinatorShard.super.onUnloaded();
+        coordinatorMetrics.deactivateMetricsShard(metricsShard);
     }
 
     @Override
     public void replay(long offset, long producerId, short producerEpoch, 
CoordinatorRecord record) throws RuntimeException {
+        ApiMessageAndVersion key = record.key();
+        ApiMessageAndVersion value = record.value();
+
+        switch (key.version()) {
+            case ShareCoordinator.SHARE_SNAPSHOT_RECORD_KEY_VERSION: // 
ShareSnapshot
+                handleShareSnapshot((ShareSnapshotKey) key.message(), 
(ShareSnapshotValue) messageOrNull(value));
+                break;
+            case ShareCoordinator.SHARE_UPDATE_RECORD_KEY_VERSION: // 
ShareUpdate
+                handleShareUpdate((ShareUpdateKey) key.message(), 
(ShareUpdateValue) messageOrNull(value));
+                break;
+            default:
+                // noop
+        }
+    }
+
+    private void handleShareSnapshot(ShareSnapshotKey key, ShareSnapshotValue 
value) {
+        SharePartitionKey mapKey = 
SharePartitionKey.getInstance(key.groupId(), key.topicId(), key.partition());
+        maybeUpdateLeaderEpochMap(mapKey, value.leaderEpoch());
+        maybeUpdateStateEpochMap(mapKey, value.stateEpoch());
+
+        ShareGroupOffset offsetRecord = ShareGroupOffset.fromRecord(value);
+        // this record is the complete snapshot
+        shareStateMap.put(mapKey, offsetRecord);
+        // if number of share updates is exceeded, then reset it
+        if (snapshotUpdateCount.containsKey(mapKey)) {
+            if (snapshotUpdateCount.get(mapKey) >= 
snapshotUpdateRecordsPerSnapshot) {
+                snapshotUpdateCount.put(mapKey, 0);
+            }
+        }
+    }
+
+    private void handleShareUpdate(ShareUpdateKey key, ShareUpdateValue value) 
{
+        SharePartitionKey mapKey = 
SharePartitionKey.getInstance(key.groupId(), key.topicId(), key.partition());
+        maybeUpdateLeaderEpochMap(mapKey, value.leaderEpoch());
+
+        // share update does not hold state epoch information.
+
+        ShareGroupOffset offsetRecord = ShareGroupOffset.fromRecord(value);
+        // this is an incremental snapshot
+        // so, we need to apply it to our current soft state
+        shareStateMap.compute(mapKey, (k, v) -> v == null ? offsetRecord : 
merge(v, value));
+        snapshotUpdateCount.compute(mapKey, (k, v) -> v == null ? 0 : v + 1);
+    }
+
+    private void maybeUpdateLeaderEpochMap(SharePartitionKey mapKey, int 
leaderEpoch) {
+        leaderEpochMap.putIfAbsent(mapKey, leaderEpoch);
+        if (leaderEpochMap.get(mapKey) < leaderEpoch) {
+            leaderEpochMap.put(mapKey, leaderEpoch);
+        }
+    }
+
+    private void maybeUpdateStateEpochMap(SharePartitionKey mapKey, int 
stateEpoch) {
+        stateEpochMap.putIfAbsent(mapKey, stateEpoch);
+        if (stateEpochMap.get(mapKey) < stateEpoch) {
+            stateEpochMap.put(mapKey, stateEpoch);
+        }
     }
 
     @Override
     public void replayEndTransactionMarker(long producerId, short 
producerEpoch, TransactionResult result) throws RuntimeException {
         CoordinatorShard.super.replayEndTransactionMarker(producerId, 
producerEpoch, result);
     }
 
-    public CoordinatorResult<WriteShareGroupStateResponseData, 
CoordinatorRecord> writeState(RequestContext context, 
WriteShareGroupStateRequestData request) {
-        throw new RuntimeException("Not implemented");
+    /**
+     * This method generates the ShareSnapshotValue record corresponding to 
the requested topic partition information.
+     * The generated record is then written to the __share_group_state topic 
and replayed to the in-memory state
+     * of the coordinator shard, shareStateMap, by CoordinatorRuntime.
+     * <p>
+     * This method as called by the ShareCoordinatorService will be provided 
with
+     * the request data which covers only a single key i.e. 
group1:topic1:partition1. The implementation
+     * below was done keeping this in mind.
+     *
+     * @param context - RequestContext
+     * @param request - WriteShareGroupStateRequestData for a single key
+     * @return CoordinatorResult(records, response)
+     */
+    @SuppressWarnings("NPathComplexity")
+    public CoordinatorResult<WriteShareGroupStateResponseData, 
CoordinatorRecord> writeState(
+        RequestContext context,
+        WriteShareGroupStateRequestData request
+    ) {
+        // records to write (with both key and value of snapshot type), 
response to caller
+        // only one key will be there in the request by design
+
+        
metricsShard.record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
+        Optional<CoordinatorResult<WriteShareGroupStateResponseData, 
CoordinatorRecord>> error = maybeGetWriteStateError(request);
+        if (error.isPresent()) {
+            return error.get();
+        }
+
+        String groupId = request.groupId();
+        WriteShareGroupStateRequestData.WriteStateData topicData = 
request.topics().get(0);
+        WriteShareGroupStateRequestData.PartitionData partitionData = 
topicData.partitions().get(0);
+
+        SharePartitionKey key = SharePartitionKey.getInstance(groupId, 
topicData.topicId(), partitionData.partition());
+        List<CoordinatorRecord> recordList;
+
+        if (!shareStateMap.containsKey(key)) {
+            // since this is the first time we are getting a write request, we 
should be creating a share snapshot record
+            recordList = 
Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
+                groupId, topicData.topicId(), partitionData.partition(), 
ShareGroupOffset.fromRequest(partitionData)
+            ));
+        } else if (snapshotUpdateCount.getOrDefault(key, 0) >= 
snapshotUpdateRecordsPerSnapshot) {
+            // Since the number of update records for this share part key 
exceeds snapshotUpdateRecordsPerSnapshot,
+            // we should be creating a share snapshot record.
+            List<PersisterOffsetsStateBatch> batchesToAdd;
+            if (partitionData.startOffset() == -1) {
+                batchesToAdd = combineStateBatches(
+                    shareStateMap.get(key).stateBatchAsSet(),
+                    partitionData.stateBatches().stream()
+                        .map(PersisterOffsetsStateBatch::from)
+                        .collect(Collectors.toCollection(LinkedHashSet::new)));
+            } else {
+                // start offset is being updated - we should only
+                // consider new updates to batches
+                batchesToAdd = partitionData.stateBatches().stream()

Review Comment:
   Fixed and added test for the scenario `testNonSequentialBatchUpdates`



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