chia7712 commented on code in PR #17011:
URL: https://github.com/apache/kafka/pull/17011#discussion_r2680957131
##########
share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java:
##########
@@ -17,58 +17,565 @@
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.requests.RequestContext;
+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.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.Collection;
+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 ShareCoordinatorConfig config;
+ private final CoordinatorMetrics coordinatorMetrics;
+ private final CoordinatorMetricsShard metricsShard;
+ private final TimelineHashMap<SharePartitionKey, ShareGroupOffset>
shareStateMap; // coord key -> ShareGroupOffset
+ // leaderEpochMap can be updated by writeState call
+ // or if a newer leader makes a readState call.
+ private final TimelineHashMap<SharePartitionKey, Integer> leaderEpochMap;
+ private final TimelineHashMap<SharePartitionKey, Integer>
snapshotUpdateCount;
+ private final TimelineHashMap<SharePartitionKey, Integer> stateEpochMap;
+ private MetadataImage metadataImage;
+
+ 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 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) {
+ // method is required due to interface
+ return this;
+ }
+
+ @Override
+ public CoordinatorShardBuilder<ShareCoordinatorShard,
CoordinatorRecord> withTimer(CoordinatorTimer<Void, CoordinatorRecord> timer) {
+ // method is required due to interface
+ 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 (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,
+ config,
+ coordinatorMetrics,
+ metricsShard,
+ snapshotRegistry
+ );
+ }
+ }
+
+ ShareCoordinatorShard(
+ LogContext logContext,
+ ShareCoordinatorConfig config,
+ CoordinatorMetrics coordinatorMetrics,
+ CoordinatorMetricsShard metricsShard,
+ SnapshotRegistry snapshotRegistry
+ ) {
+ this.log = logContext.logger(ShareCoordinatorShard.class);
+ 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);
+ }
+
@Override
public void onLoaded(MetadataImage newImage) {
- CoordinatorShard.super.onLoaded(newImage);
+ this.metadataImage = 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) >=
config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot()) {
+ 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);
Review Comment:
I will file a patch if it gets confirmed :)
--
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]