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


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -4438,18 +4443,46 @@ class KafkaApis(val requestChannel: RequestChannel,
     CompletableFuture.completedFuture[Unit](())
   }
 
-  def handleReadShareGroupStateRequest(request: RequestChannel.Request): Unit 
= {
+  def handleReadShareGroupStateRequest(request: RequestChannel.Request): 
CompletableFuture[Unit] = {
     val readShareGroupStateRequest = request.body[ReadShareGroupStateRequest]
-    // TODO: Implement the ReadShareGroupStateRequest handling
-    requestHelper.sendMaybeThrottle(request, 
readShareGroupStateRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
-    CompletableFuture.completedFuture[Unit](())
+
+    authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    shareCoordinator match {
+      case None => requestHelper.sendResponseMaybeThrottle(request, 
requestThrottleMs =>
+        readShareGroupStateRequest.getErrorResponse(requestThrottleMs,
+          new ApiException("Share coordinator is not configured.")))

Review Comment:
   I would say that's an internal server error. Only Kafka code issues this 
RPC. It only does it when the share coordinator is enabled.



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -4438,18 +4443,46 @@ class KafkaApis(val requestChannel: RequestChannel,
     CompletableFuture.completedFuture[Unit](())
   }
 
-  def handleReadShareGroupStateRequest(request: RequestChannel.Request): Unit 
= {
+  def handleReadShareGroupStateRequest(request: RequestChannel.Request): 
CompletableFuture[Unit] = {
     val readShareGroupStateRequest = request.body[ReadShareGroupStateRequest]
-    // TODO: Implement the ReadShareGroupStateRequest handling
-    requestHelper.sendMaybeThrottle(request, 
readShareGroupStateRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
-    CompletableFuture.completedFuture[Unit](())
+
+    authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    shareCoordinator match {
+      case None => requestHelper.sendResponseMaybeThrottle(request, 
requestThrottleMs =>
+        readShareGroupStateRequest.getErrorResponse(requestThrottleMs,
+          new ApiException("Share coordinator is not configured.")))
+        CompletableFuture.completedFuture[Unit](())
+      case Some(coordinator) => coordinator.readState(request.context, 
readShareGroupStateRequest.data)
+        .handle[Unit] { (response, exception) =>
+          if (exception != null) {
+            requestHelper.sendMaybeThrottle(request, 
readShareGroupStateRequest.getErrorResponse(exception))
+          } else {
+            requestHelper.sendMaybeThrottle(request, new 
ReadShareGroupStateResponse(response))
+          }
+        }
+    }
   }
 
-  def handleWriteShareGroupStateRequest(request: RequestChannel.Request): Unit 
= {
-    val writeShareGroupStateRequest = request.body[WriteShareGroupStateRequest]
-    // TODO: Implement the WriteShareGroupStateRequest handling
-    requestHelper.sendMaybeThrottle(request, 
writeShareGroupStateRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
-    CompletableFuture.completedFuture[Unit](())
+  def handleWriteShareGroupStateRequest(request: RequestChannel.Request): 
CompletableFuture[Unit] = {
+    val writeShareRequest = request.body[WriteShareGroupStateRequest]
+
+    authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    shareCoordinator match {
+      case None => requestHelper.sendResponseMaybeThrottle(request, 
requestThrottleMs =>
+        writeShareRequest.getErrorResponse(requestThrottleMs,
+          new ApiException("Share coordinator is not configured.")))

Review Comment:
   And this one.



##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java:
##########
@@ -0,0 +1,523 @@
+/*
+ * 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.share;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.compress.Compression;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.internals.Topic;
+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.Errors;
+import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
+import org.apache.kafka.common.requests.RequestContext;
+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.common.utils.Utils;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorEventProcessor;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetrics;
+import 
org.apache.kafka.coordinator.common.runtime.CoordinatorShardBuilderSupplier;
+import org.apache.kafka.coordinator.common.runtime.MultiThreadedEventProcessor;
+import org.apache.kafka.coordinator.common.runtime.PartitionWriter;
+import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
+import org.apache.kafka.server.config.ShareCoordinatorConfig;
+import org.apache.kafka.server.group.share.SharePartitionKey;
+import org.apache.kafka.server.record.BrokerCompressionType;
+import org.apache.kafka.server.util.timer.Timer;
+
+import org.slf4j.Logger;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+
+public class ShareCoordinatorService implements ShareCoordinator {
+    private final ShareCoordinatorConfig config;
+    private final Logger log;
+    private final AtomicBoolean isActive = new AtomicBoolean(false);  // for 
controlling start and stop
+    private final CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> 
runtime;
+    private final ShareCoordinatorMetrics shareCoordinatorMetrics;
+    private volatile int numPartitions = -1; // Number of partitions for 
__share_group_state. Provided when component is started.
+    private final Time time;
+
+    public static class Builder {
+        private final int nodeId;
+        private final ShareCoordinatorConfig config;
+        private PartitionWriter writer;
+        private CoordinatorLoader<CoordinatorRecord> loader;
+        private Time time;
+        private Timer timer;
+
+        private ShareCoordinatorMetrics coordinatorMetrics;
+        private CoordinatorRuntimeMetrics coordinatorRuntimeMetrics;
+
+        public Builder(int nodeId, ShareCoordinatorConfig config) {
+            this.nodeId = nodeId;
+            this.config = config;
+        }
+
+        public Builder withWriter(PartitionWriter writer) {
+            this.writer = writer;
+            return this;
+        }
+
+        public Builder withLoader(CoordinatorLoader<CoordinatorRecord> loader) 
{
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder withTimer(Timer timer) {
+            this.timer = timer;
+            return this;
+        }
+
+        public Builder withCoordinatorMetrics(ShareCoordinatorMetrics 
coordinatorMetrics) {
+            this.coordinatorMetrics = coordinatorMetrics;
+            return this;
+        }
+
+        public Builder withCoordinatorRuntimeMetrics(CoordinatorRuntimeMetrics 
coordinatorRuntimeMetrics) {
+            this.coordinatorRuntimeMetrics = coordinatorRuntimeMetrics;
+            return this;
+        }
+
+        public ShareCoordinatorService build() {
+            if (config == null) {
+                throw new IllegalArgumentException("Config must be set.");
+            }
+            if (writer == null) {
+                throw new IllegalArgumentException("Writer must be set.");
+            }
+            if (loader == null) {
+                throw new IllegalArgumentException("Loader 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) {
+                throw new IllegalArgumentException("Share Coordinator metrics 
must be set.");
+            }
+            if (coordinatorRuntimeMetrics == null) {
+                throw new IllegalArgumentException("Coordinator runtime 
metrics must be set.");
+            }
+
+            String logPrefix = String.format("ShareCoordinator id=%d", nodeId);
+            LogContext logContext = new LogContext(String.format("[%s] ", 
logPrefix));
+
+            CoordinatorShardBuilderSupplier<ShareCoordinatorShard, 
CoordinatorRecord> supplier = () ->
+                new ShareCoordinatorShard.Builder(config);
+
+            CoordinatorEventProcessor processor = new 
MultiThreadedEventProcessor(
+                logContext,
+                "share-coordinator-event-processor-",
+                config.shareCoordinatorNumThreads(),
+                time,
+                coordinatorRuntimeMetrics
+            );
+
+            CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> 
runtime =
+                new CoordinatorRuntime.Builder<ShareCoordinatorShard, 
CoordinatorRecord>()
+                    .withTime(time)
+                    .withTimer(timer)
+                    .withLogPrefix(logPrefix)
+                    .withLogContext(logContext)
+                    .withEventProcessor(processor)
+                    .withPartitionWriter(writer)
+                    .withLoader(loader)
+                    .withCoordinatorShardBuilderSupplier(supplier)
+                    .withTime(time)
+                    
.withDefaultWriteTimeOut(Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()))
+                    .withCoordinatorRuntimeMetrics(coordinatorRuntimeMetrics)
+                    .withCoordinatorMetrics(coordinatorMetrics)
+                    .withSerializer(new ShareCoordinatorRecordSerde())
+                    
.withCompression(Compression.of(config.shareCoordinatorStateTopicCompressionType()).build())
+                    
.withAppendLingerMs(config.shareCoordinatorAppendLingerMs())
+                    .build();
+
+            return new ShareCoordinatorService(
+                logContext,
+                config,
+                runtime,
+                coordinatorMetrics,
+                time
+            );
+        }
+    }
+
+    public ShareCoordinatorService(
+        LogContext logContext,
+        ShareCoordinatorConfig config,
+        CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime,
+        ShareCoordinatorMetrics shareCoordinatorMetrics,
+        Time time) {
+        this.log = logContext.logger(ShareCoordinatorService.class);
+        this.config = config;
+        this.runtime = runtime;
+        this.shareCoordinatorMetrics = shareCoordinatorMetrics;
+        this.time = time;
+    }
+
+    @Override
+    public int partitionFor(String key) {
+        return Utils.abs(key.hashCode()) % numPartitions;
+    }
+
+    @Override
+    public Properties shareGroupStateTopicConfigs() {
+        Properties properties = new Properties();
+        properties.put(TopicConfig.CLEANUP_POLICY_CONFIG, 
TopicConfig.CLEANUP_POLICY_DELETE); // as defined in KIP-932
+        properties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, 
BrokerCompressionType.PRODUCER.name);
+        properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, 
config.shareCoordinatorStateTopicSegmentBytes());
+        return properties;
+    }
+
+    @Override
+    public void startup(
+        IntSupplier shareGroupTopicPartitionCount
+    ) {
+        if (!isActive.compareAndSet(false, true)) {
+            log.warn("Share coordinator is already running.");
+            return;
+        }
+
+        log.info("Starting up.");
+        numPartitions = shareGroupTopicPartitionCount.getAsInt();
+        isActive.set(true);
+        log.info("Startup complete.");
+    }
+
+    @Override
+    public void shutdown() {
+        if (!isActive.compareAndSet(true, false)) {
+            log.warn("Share coordinator is already shutting down.");
+            return;
+        }
+
+        log.info("Shutting down.");
+        isActive.set(false);
+        Utils.closeQuietly(runtime, "coordinator runtime");
+        Utils.closeQuietly(shareCoordinatorMetrics, "share coordinator 
metrics");
+        log.info("Shutdown complete.");
+    }
+
+    @Override
+    public CompletableFuture<WriteShareGroupStateResponseData> 
writeState(RequestContext context, WriteShareGroupStateRequestData request) {
+        log.debug("ShareCoordinatorService writeState request dump - {}", 
request);
+
+        // Send an empty response if topic data is empty
+        if (isEmpty(request.topics())) {
+            log.error("Topic Data is empty: {}", request);
+            return CompletableFuture.completedFuture(
+                new WriteShareGroupStateResponseData()
+            );
+        }
+
+        // Send an empty response if partition data is empty for any topic
+        for (WriteShareGroupStateRequestData.WriteStateData topicData : 
request.topics()) {
+            if (isEmpty(topicData.partitions())) {
+                log.error("Partition Data for topic {} is empty: {}", 
topicData.topicId(), request);
+                return CompletableFuture.completedFuture(
+                    new WriteShareGroupStateResponseData()
+                );
+            }
+        }
+
+        String groupId = request.groupId();
+        // Send an empty response if groupId is invalid
+        if (isGroupIdEmpty(groupId)) {
+            log.error("Group id must be specified and non-empty: {}", request);

Review Comment:
   Probably INVALID_REQUEST if it's empty.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java:
##########
@@ -150,6 +159,34 @@ public GroupCoordinatorMetrics(MetricsRegistry registry, 
Metrics metrics) {
             Collections.singletonMap(CONSUMER_GROUP_COUNT_STATE_TAG, 
ConsumerGroupState.DEAD.toString())
         );
 
+        shareGroupCountMetricName = metrics.metricName(
+            SHARE_GROUP_COUNT_METRIC_NAME,

Review Comment:
   These metric names do not match the KIP. I am certainly happy to tweak the 
KIP, but either way the KIP and the code need to match. Let me know what you 
want to do here @smjn .



##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java:
##########
@@ -0,0 +1,523 @@
+/*
+ * 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.share;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.compress.Compression;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.internals.Topic;
+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.Errors;
+import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
+import org.apache.kafka.common.requests.RequestContext;
+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.common.utils.Utils;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorEventProcessor;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetrics;
+import 
org.apache.kafka.coordinator.common.runtime.CoordinatorShardBuilderSupplier;
+import org.apache.kafka.coordinator.common.runtime.MultiThreadedEventProcessor;
+import org.apache.kafka.coordinator.common.runtime.PartitionWriter;
+import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
+import org.apache.kafka.server.config.ShareCoordinatorConfig;
+import org.apache.kafka.server.group.share.SharePartitionKey;
+import org.apache.kafka.server.record.BrokerCompressionType;
+import org.apache.kafka.server.util.timer.Timer;
+
+import org.slf4j.Logger;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+
+public class ShareCoordinatorService implements ShareCoordinator {
+    private final ShareCoordinatorConfig config;
+    private final Logger log;
+    private final AtomicBoolean isActive = new AtomicBoolean(false);  // for 
controlling start and stop
+    private final CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> 
runtime;
+    private final ShareCoordinatorMetrics shareCoordinatorMetrics;
+    private volatile int numPartitions = -1; // Number of partitions for 
__share_group_state. Provided when component is started.
+    private final Time time;
+
+    public static class Builder {
+        private final int nodeId;
+        private final ShareCoordinatorConfig config;
+        private PartitionWriter writer;
+        private CoordinatorLoader<CoordinatorRecord> loader;
+        private Time time;
+        private Timer timer;
+
+        private ShareCoordinatorMetrics coordinatorMetrics;
+        private CoordinatorRuntimeMetrics coordinatorRuntimeMetrics;
+
+        public Builder(int nodeId, ShareCoordinatorConfig config) {
+            this.nodeId = nodeId;
+            this.config = config;
+        }
+
+        public Builder withWriter(PartitionWriter writer) {
+            this.writer = writer;
+            return this;
+        }
+
+        public Builder withLoader(CoordinatorLoader<CoordinatorRecord> loader) 
{
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder withTimer(Timer timer) {
+            this.timer = timer;
+            return this;
+        }
+
+        public Builder withCoordinatorMetrics(ShareCoordinatorMetrics 
coordinatorMetrics) {
+            this.coordinatorMetrics = coordinatorMetrics;
+            return this;
+        }
+
+        public Builder withCoordinatorRuntimeMetrics(CoordinatorRuntimeMetrics 
coordinatorRuntimeMetrics) {
+            this.coordinatorRuntimeMetrics = coordinatorRuntimeMetrics;
+            return this;
+        }
+
+        public ShareCoordinatorService build() {
+            if (config == null) {
+                throw new IllegalArgumentException("Config must be set.");
+            }
+            if (writer == null) {
+                throw new IllegalArgumentException("Writer must be set.");
+            }
+            if (loader == null) {
+                throw new IllegalArgumentException("Loader 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) {
+                throw new IllegalArgumentException("Share Coordinator metrics 
must be set.");
+            }
+            if (coordinatorRuntimeMetrics == null) {
+                throw new IllegalArgumentException("Coordinator runtime 
metrics must be set.");
+            }
+
+            String logPrefix = String.format("ShareCoordinator id=%d", nodeId);
+            LogContext logContext = new LogContext(String.format("[%s] ", 
logPrefix));
+
+            CoordinatorShardBuilderSupplier<ShareCoordinatorShard, 
CoordinatorRecord> supplier = () ->
+                new ShareCoordinatorShard.Builder(config);
+
+            CoordinatorEventProcessor processor = new 
MultiThreadedEventProcessor(
+                logContext,
+                "share-coordinator-event-processor-",
+                config.shareCoordinatorNumThreads(),
+                time,
+                coordinatorRuntimeMetrics
+            );
+
+            CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> 
runtime =
+                new CoordinatorRuntime.Builder<ShareCoordinatorShard, 
CoordinatorRecord>()
+                    .withTime(time)
+                    .withTimer(timer)
+                    .withLogPrefix(logPrefix)
+                    .withLogContext(logContext)
+                    .withEventProcessor(processor)
+                    .withPartitionWriter(writer)
+                    .withLoader(loader)
+                    .withCoordinatorShardBuilderSupplier(supplier)
+                    .withTime(time)
+                    
.withDefaultWriteTimeOut(Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()))
+                    .withCoordinatorRuntimeMetrics(coordinatorRuntimeMetrics)
+                    .withCoordinatorMetrics(coordinatorMetrics)
+                    .withSerializer(new ShareCoordinatorRecordSerde())
+                    
.withCompression(Compression.of(config.shareCoordinatorStateTopicCompressionType()).build())
+                    
.withAppendLingerMs(config.shareCoordinatorAppendLingerMs())
+                    .build();
+
+            return new ShareCoordinatorService(
+                logContext,
+                config,
+                runtime,
+                coordinatorMetrics,
+                time
+            );
+        }
+    }
+
+    public ShareCoordinatorService(
+        LogContext logContext,
+        ShareCoordinatorConfig config,
+        CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime,
+        ShareCoordinatorMetrics shareCoordinatorMetrics,
+        Time time) {
+        this.log = logContext.logger(ShareCoordinatorService.class);
+        this.config = config;
+        this.runtime = runtime;
+        this.shareCoordinatorMetrics = shareCoordinatorMetrics;
+        this.time = time;
+    }
+
+    @Override
+    public int partitionFor(String key) {
+        return Utils.abs(key.hashCode()) % numPartitions;
+    }
+
+    @Override
+    public Properties shareGroupStateTopicConfigs() {
+        Properties properties = new Properties();
+        properties.put(TopicConfig.CLEANUP_POLICY_CONFIG, 
TopicConfig.CLEANUP_POLICY_DELETE); // as defined in KIP-932
+        properties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, 
BrokerCompressionType.PRODUCER.name);
+        properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, 
config.shareCoordinatorStateTopicSegmentBytes());
+        return properties;
+    }
+
+    @Override
+    public void startup(
+        IntSupplier shareGroupTopicPartitionCount
+    ) {
+        if (!isActive.compareAndSet(false, true)) {
+            log.warn("Share coordinator is already running.");
+            return;
+        }
+
+        log.info("Starting up.");
+        numPartitions = shareGroupTopicPartitionCount.getAsInt();
+        isActive.set(true);
+        log.info("Startup complete.");
+    }
+
+    @Override
+    public void shutdown() {
+        if (!isActive.compareAndSet(true, false)) {
+            log.warn("Share coordinator is already shutting down.");
+            return;
+        }
+
+        log.info("Shutting down.");
+        isActive.set(false);
+        Utils.closeQuietly(runtime, "coordinator runtime");
+        Utils.closeQuietly(shareCoordinatorMetrics, "share coordinator 
metrics");
+        log.info("Shutdown complete.");
+    }
+
+    @Override
+    public CompletableFuture<WriteShareGroupStateResponseData> 
writeState(RequestContext context, WriteShareGroupStateRequestData request) {
+        log.debug("ShareCoordinatorService writeState request dump - {}", 
request);
+
+        // Send an empty response if topic data is empty
+        if (isEmpty(request.topics())) {
+            log.error("Topic Data is empty: {}", request);
+            return CompletableFuture.completedFuture(
+                new WriteShareGroupStateResponseData()
+            );
+        }
+
+        // Send an empty response if partition data is empty for any topic
+        for (WriteShareGroupStateRequestData.WriteStateData topicData : 
request.topics()) {
+            if (isEmpty(topicData.partitions())) {
+                log.error("Partition Data for topic {} is empty: {}", 
topicData.topicId(), request);
+                return CompletableFuture.completedFuture(
+                    new WriteShareGroupStateResponseData()
+                );
+            }
+        }
+
+        String groupId = request.groupId();
+        // Send an empty response if groupId is invalid
+        if (isGroupIdEmpty(groupId)) {
+            log.error("Group id must be specified and non-empty: {}", request);
+            return CompletableFuture.completedFuture(
+                new WriteShareGroupStateResponseData()
+            );
+        }
+
+        // Send an empty response if the coordinator is not active
+        if (!isActive.get()) {
+            return CompletableFuture.completedFuture(
+                generateErrorWriteStateResponse(
+                    request,
+                    Errors.COORDINATOR_NOT_AVAILABLE,
+                    "Share coordinator is not available."
+                )
+            );
+        }
+
+        // The request received here could have multiple keys of structure 
group:topic:partition. However,
+        // the writeState method in ShareCoordinatorShard expects a single key 
in the request. Hence, we will
+        // be looping over the keys below and constructing new 
WriteShareGroupStateRequestData objects to pass
+        // onto the shard method.
+        Map<Uuid, Map<Integer, 
CompletableFuture<WriteShareGroupStateResponseData>>> futureMap = new 
HashMap<>();
+        long startTime = time.hiResClockMs();
+
+        request.topics().forEach(topicData -> {
+            Map<Integer, CompletableFuture<WriteShareGroupStateResponseData>> 
partitionFut =
+                futureMap.computeIfAbsent(topicData.topicId(), k -> new 
HashMap<>());
+            topicData.partitions().forEach(
+                partitionData -> partitionFut.put(partitionData.partition(), 
runtime.scheduleWriteOperation(
+                    "write-share-group-state",
+                    topicPartitionFor(SharePartitionKey.getInstance(groupId, 
topicData.topicId(), partitionData.partition())),
+                    Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()),
+                    coordinator -> coordinator.writeState(context, new 
WriteShareGroupStateRequestData()
+                        .setGroupId(groupId)
+                        .setTopics(Collections.singletonList(new 
WriteShareGroupStateRequestData.WriteStateData()
+                            .setTopicId(topicData.topicId())
+                            .setPartitions(Collections.singletonList(new 
WriteShareGroupStateRequestData.PartitionData()
+                                .setPartition(partitionData.partition())
+                                .setStartOffset(partitionData.startOffset())
+                                .setLeaderEpoch(partitionData.leaderEpoch())
+                                .setStateEpoch(partitionData.stateEpoch())
+                                
.setStateBatches(partitionData.stateBatches()))))))))
+            );
+        });
+
+        // Combine all futures into a single CompletableFuture<Void>
+        CompletableFuture<Void> combinedFuture = 
CompletableFuture.allOf(futureMap.values().stream()
+            .flatMap(partMap -> 
partMap.values().stream()).toArray(CompletableFuture[]::new));
+
+        // topicId -> {partitionId -> responseFuture}
+        return combinedFuture.thenApply(v -> {
+            List<WriteShareGroupStateResponseData.WriteStateResult> 
writeStateResults = new LinkedList<>();
+            futureMap.forEach(
+                (topicId, topicEntry) -> {
+                    List<WriteShareGroupStateResponseData.PartitionResult> 
partitionResults = new LinkedList<>();
+                    topicEntry.forEach(
+                        // map of partition id -> responses from api
+                        (partitionId, responseFut) -> {
+                            try {
+                                long timeTaken = time.hiResClockMs() - 
startTime;
+                                // This is the future returned by 
runtime.scheduleWriteOperation which returns when the
+                                // operation has completed including
+                                WriteShareGroupStateResponseData partitionData 
= responseFut.get(5000L, TimeUnit.MILLISECONDS);

Review Comment:
   Doesn't this only run when the inidividual futures are complete? I'm 
surprised you think you need to wait up to 5 seconds here.



##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java:
##########
@@ -17,58 +17,553 @@
 
 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 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.");
+            if (config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot() < 0 
|| config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot() > 500)
+                throw new 
IllegalArgumentException("SnapshotUpdateRecordsPerSnapshot must be between 0 
and 500.");
+
+            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));

Review Comment:
   In the future when we delete these things, I expect we'll write tombstones 
as markers even though it's not a compacted topic. This enables us to bookkeep 
the records on the topic and work out what we can prune.



##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java:
##########
@@ -0,0 +1,523 @@
+/*
+ * 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.share;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.compress.Compression;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.internals.Topic;
+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.Errors;
+import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
+import org.apache.kafka.common.requests.RequestContext;
+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.common.utils.Utils;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorEventProcessor;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetrics;
+import 
org.apache.kafka.coordinator.common.runtime.CoordinatorShardBuilderSupplier;
+import org.apache.kafka.coordinator.common.runtime.MultiThreadedEventProcessor;
+import org.apache.kafka.coordinator.common.runtime.PartitionWriter;
+import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
+import org.apache.kafka.server.config.ShareCoordinatorConfig;
+import org.apache.kafka.server.group.share.SharePartitionKey;
+import org.apache.kafka.server.record.BrokerCompressionType;
+import org.apache.kafka.server.util.timer.Timer;
+
+import org.slf4j.Logger;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+
+public class ShareCoordinatorService implements ShareCoordinator {
+    private final ShareCoordinatorConfig config;
+    private final Logger log;
+    private final AtomicBoolean isActive = new AtomicBoolean(false);  // for 
controlling start and stop
+    private final CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> 
runtime;
+    private final ShareCoordinatorMetrics shareCoordinatorMetrics;
+    private volatile int numPartitions = -1; // Number of partitions for 
__share_group_state. Provided when component is started.
+    private final Time time;
+
+    public static class Builder {
+        private final int nodeId;
+        private final ShareCoordinatorConfig config;
+        private PartitionWriter writer;
+        private CoordinatorLoader<CoordinatorRecord> loader;
+        private Time time;
+        private Timer timer;
+
+        private ShareCoordinatorMetrics coordinatorMetrics;
+        private CoordinatorRuntimeMetrics coordinatorRuntimeMetrics;
+
+        public Builder(int nodeId, ShareCoordinatorConfig config) {
+            this.nodeId = nodeId;
+            this.config = config;
+        }
+
+        public Builder withWriter(PartitionWriter writer) {
+            this.writer = writer;
+            return this;
+        }
+
+        public Builder withLoader(CoordinatorLoader<CoordinatorRecord> loader) 
{
+            this.loader = loader;
+            return this;
+        }
+
+        public Builder withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder withTimer(Timer timer) {
+            this.timer = timer;
+            return this;
+        }
+
+        public Builder withCoordinatorMetrics(ShareCoordinatorMetrics 
coordinatorMetrics) {
+            this.coordinatorMetrics = coordinatorMetrics;
+            return this;
+        }
+
+        public Builder withCoordinatorRuntimeMetrics(CoordinatorRuntimeMetrics 
coordinatorRuntimeMetrics) {
+            this.coordinatorRuntimeMetrics = coordinatorRuntimeMetrics;
+            return this;
+        }
+
+        public ShareCoordinatorService build() {
+            if (config == null) {
+                throw new IllegalArgumentException("Config must be set.");
+            }
+            if (writer == null) {
+                throw new IllegalArgumentException("Writer must be set.");
+            }
+            if (loader == null) {
+                throw new IllegalArgumentException("Loader 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) {
+                throw new IllegalArgumentException("Share Coordinator metrics 
must be set.");
+            }
+            if (coordinatorRuntimeMetrics == null) {
+                throw new IllegalArgumentException("Coordinator runtime 
metrics must be set.");
+            }
+
+            String logPrefix = String.format("ShareCoordinator id=%d", nodeId);
+            LogContext logContext = new LogContext(String.format("[%s] ", 
logPrefix));
+
+            CoordinatorShardBuilderSupplier<ShareCoordinatorShard, 
CoordinatorRecord> supplier = () ->
+                new ShareCoordinatorShard.Builder(config);
+
+            CoordinatorEventProcessor processor = new 
MultiThreadedEventProcessor(
+                logContext,
+                "share-coordinator-event-processor-",
+                config.shareCoordinatorNumThreads(),
+                time,
+                coordinatorRuntimeMetrics
+            );
+
+            CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> 
runtime =
+                new CoordinatorRuntime.Builder<ShareCoordinatorShard, 
CoordinatorRecord>()
+                    .withTime(time)
+                    .withTimer(timer)
+                    .withLogPrefix(logPrefix)
+                    .withLogContext(logContext)
+                    .withEventProcessor(processor)
+                    .withPartitionWriter(writer)
+                    .withLoader(loader)
+                    .withCoordinatorShardBuilderSupplier(supplier)
+                    .withTime(time)
+                    
.withDefaultWriteTimeOut(Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()))
+                    .withCoordinatorRuntimeMetrics(coordinatorRuntimeMetrics)
+                    .withCoordinatorMetrics(coordinatorMetrics)
+                    .withSerializer(new ShareCoordinatorRecordSerde())
+                    
.withCompression(Compression.of(config.shareCoordinatorStateTopicCompressionType()).build())
+                    
.withAppendLingerMs(config.shareCoordinatorAppendLingerMs())
+                    .build();
+
+            return new ShareCoordinatorService(
+                logContext,
+                config,
+                runtime,
+                coordinatorMetrics,
+                time
+            );
+        }
+    }
+
+    public ShareCoordinatorService(
+        LogContext logContext,
+        ShareCoordinatorConfig config,
+        CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime,
+        ShareCoordinatorMetrics shareCoordinatorMetrics,
+        Time time) {
+        this.log = logContext.logger(ShareCoordinatorService.class);
+        this.config = config;
+        this.runtime = runtime;
+        this.shareCoordinatorMetrics = shareCoordinatorMetrics;
+        this.time = time;
+    }
+
+    @Override
+    public int partitionFor(String key) {
+        return Utils.abs(key.hashCode()) % numPartitions;
+    }
+
+    @Override
+    public Properties shareGroupStateTopicConfigs() {
+        Properties properties = new Properties();
+        properties.put(TopicConfig.CLEANUP_POLICY_CONFIG, 
TopicConfig.CLEANUP_POLICY_DELETE); // as defined in KIP-932
+        properties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, 
BrokerCompressionType.PRODUCER.name);
+        properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, 
config.shareCoordinatorStateTopicSegmentBytes());
+        return properties;
+    }
+
+    @Override
+    public void startup(
+        IntSupplier shareGroupTopicPartitionCount
+    ) {
+        if (!isActive.compareAndSet(false, true)) {
+            log.warn("Share coordinator is already running.");
+            return;
+        }
+
+        log.info("Starting up.");
+        numPartitions = shareGroupTopicPartitionCount.getAsInt();
+        isActive.set(true);
+        log.info("Startup complete.");
+    }
+
+    @Override
+    public void shutdown() {
+        if (!isActive.compareAndSet(true, false)) {
+            log.warn("Share coordinator is already shutting down.");
+            return;
+        }
+
+        log.info("Shutting down.");
+        isActive.set(false);
+        Utils.closeQuietly(runtime, "coordinator runtime");
+        Utils.closeQuietly(shareCoordinatorMetrics, "share coordinator 
metrics");
+        log.info("Shutdown complete.");
+    }
+
+    @Override
+    public CompletableFuture<WriteShareGroupStateResponseData> 
writeState(RequestContext context, WriteShareGroupStateRequestData request) {
+        log.debug("ShareCoordinatorService writeState request dump - {}", 
request);
+
+        // Send an empty response if topic data is empty
+        if (isEmpty(request.topics())) {
+            log.error("Topic Data is empty: {}", request);
+            return CompletableFuture.completedFuture(
+                new WriteShareGroupStateResponseData()
+            );
+        }
+
+        // Send an empty response if partition data is empty for any topic
+        for (WriteShareGroupStateRequestData.WriteStateData topicData : 
request.topics()) {
+            if (isEmpty(topicData.partitions())) {
+                log.error("Partition Data for topic {} is empty: {}", 
topicData.topicId(), request);
+                return CompletableFuture.completedFuture(
+                    new WriteShareGroupStateResponseData()
+                );
+            }
+        }
+
+        String groupId = request.groupId();
+        // Send an empty response if groupId is invalid
+        if (isGroupIdEmpty(groupId)) {
+            log.error("Group id must be specified and non-empty: {}", request);
+            return CompletableFuture.completedFuture(
+                new WriteShareGroupStateResponseData()
+            );
+        }
+
+        // Send an empty response if the coordinator is not active
+        if (!isActive.get()) {
+            return CompletableFuture.completedFuture(
+                generateErrorWriteStateResponse(
+                    request,
+                    Errors.COORDINATOR_NOT_AVAILABLE,
+                    "Share coordinator is not available."
+                )
+            );
+        }
+
+        // The request received here could have multiple keys of structure 
group:topic:partition. However,
+        // the writeState method in ShareCoordinatorShard expects a single key 
in the request. Hence, we will
+        // be looping over the keys below and constructing new 
WriteShareGroupStateRequestData objects to pass
+        // onto the shard method.
+        Map<Uuid, Map<Integer, 
CompletableFuture<WriteShareGroupStateResponseData>>> futureMap = new 
HashMap<>();
+        long startTime = time.hiResClockMs();
+
+        request.topics().forEach(topicData -> {
+            Map<Integer, CompletableFuture<WriteShareGroupStateResponseData>> 
partitionFut =
+                futureMap.computeIfAbsent(topicData.topicId(), k -> new 
HashMap<>());
+            topicData.partitions().forEach(
+                partitionData -> partitionFut.put(partitionData.partition(), 
runtime.scheduleWriteOperation(
+                    "write-share-group-state",
+                    topicPartitionFor(SharePartitionKey.getInstance(groupId, 
topicData.topicId(), partitionData.partition())),
+                    Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()),
+                    coordinator -> coordinator.writeState(context, new 
WriteShareGroupStateRequestData()
+                        .setGroupId(groupId)
+                        .setTopics(Collections.singletonList(new 
WriteShareGroupStateRequestData.WriteStateData()
+                            .setTopicId(topicData.topicId())
+                            .setPartitions(Collections.singletonList(new 
WriteShareGroupStateRequestData.PartitionData()
+                                .setPartition(partitionData.partition())
+                                .setStartOffset(partitionData.startOffset())
+                                .setLeaderEpoch(partitionData.leaderEpoch())
+                                .setStateEpoch(partitionData.stateEpoch())
+                                
.setStateBatches(partitionData.stateBatches()))))))))
+            );
+        });
+
+        // Combine all futures into a single CompletableFuture<Void>
+        CompletableFuture<Void> combinedFuture = 
CompletableFuture.allOf(futureMap.values().stream()
+            .flatMap(partMap -> 
partMap.values().stream()).toArray(CompletableFuture[]::new));
+
+        // topicId -> {partitionId -> responseFuture}
+        return combinedFuture.thenApply(v -> {
+            List<WriteShareGroupStateResponseData.WriteStateResult> 
writeStateResults = new LinkedList<>();
+            futureMap.forEach(
+                (topicId, topicEntry) -> {
+                    List<WriteShareGroupStateResponseData.PartitionResult> 
partitionResults = new LinkedList<>();
+                    topicEntry.forEach(
+                        // map of partition id -> responses from api
+                        (partitionId, responseFut) -> {
+                            try {
+                                long timeTaken = time.hiResClockMs() - 
startTime;
+                                // This is the future returned by 
runtime.scheduleWriteOperation which returns when the
+                                // operation has completed including
+                                WriteShareGroupStateResponseData partitionData 
= responseFut.get(5000L, TimeUnit.MILLISECONDS);
+                                
shareCoordinatorMetrics.record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME,
 timeTaken);
+                                
partitionResults.addAll(partitionData.results().get(0).partitions());
+                            } catch (Exception e) {

Review Comment:
   Is there a reason why you're not using 
`GroupCoordinatorService.handleOperationException`?



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

Reply via email to