lucasbru commented on code in PR #18809:
URL: https://github.com/apache/kafka/pull/18809#discussion_r1949131601


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -3405,6 +3616,49 @@ private void scheduleConsumerGroupRebalanceTimeout(
         });
     }
 
+    /**
+     * Schedules a rebalance timeout for the member.
+     *
+     * @param groupId               The group id.
+     * @param memberId              The member id.
+     * @param memberEpoch           The member epoch.
+     * @param rebalanceTimeoutMs    The rebalance timeout.
+     */
+    private void scheduleStreamsGroupRebalanceTimeout(
+        String groupId,
+        String memberId,
+        int memberEpoch,
+        int rebalanceTimeoutMs
+    ) {
+        String key = streamsGroupRebalanceTimeoutKey(groupId, memberId);
+        timer.schedule(key, rebalanceTimeoutMs, TimeUnit.MILLISECONDS, true, 
() -> {
+            try {
+                StreamsGroup group = streamsGroup(groupId);
+                StreamsGroupMember member = 
group.getOrMaybeCreateMember(memberId, false);
+
+                if (member.memberEpoch() == memberEpoch) {
+                    log.info("[GroupId {}] Member {} fenced from the group 
because " +
+                            "it failed to transition from epoch {} within 
{}ms.",
+                        groupId, memberId, memberEpoch, rebalanceTimeoutMs);
+
+                    return streamsGroupFenceMember(group, member, null);
+                } else {
+                    log.debug("[GroupId {}] Ignoring rebalance timeout for {} 
because the member " +
+                        "left the epoch {}.", groupId, memberId, memberEpoch);
+                    return new CoordinatorResult<>(Collections.emptyList());
+                }
+            } catch (GroupIdNotFoundException ex) {
+                log.debug("[GroupId {}] Could not fence {}} because the group 
does not exist.",

Review Comment:
   Looking at the current implementation, I think timers aren't necessarily 
canceled when a group is deleted. We'd have to make sure that timers are always 
cancelled when a group is deleted. Would that be worth it? 



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -440,6 +468,7 @@ private GroupMetadataManager(
         this.groupsByTopics = new TimelineHashMap<>(snapshotRegistry, 0);
         this.groupConfigManager = groupConfigManager;
         this.shareGroupAssignor = shareGroupAssignor;
+        this.streamsGroupSessionTimeoutMs = 45000;

Review Comment:
   This is the default for consumer groups. We have a separate ticket to make 
it configurable, I just hardcoded this as a placeholder.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java:
##########
@@ -0,0 +1,1180 @@
+/*
+ * 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.streams;
+
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.StaleMemberEpochException;
+import org.apache.kafka.common.errors.UnknownMemberIdException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.StreamsGroupDescribeResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.JoinGroupRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
+import org.apache.kafka.coordinator.group.Group;
+import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
+import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
+import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
+import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
+import org.apache.kafka.coordinator.group.streams.topics.InternalTopicManager;
+import org.apache.kafka.image.ClusterImage;
+import org.apache.kafka.image.TopicImage;
+import org.apache.kafka.image.TopicsImage;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.apache.kafka.timeline.TimelineInteger;
+import org.apache.kafka.timeline.TimelineObject;
+
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.ASSIGNING;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.EMPTY;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.NOT_READY;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.RECONCILING;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.STABLE;
+
+/**
+ * A Streams Group. All the metadata in this class are backed by records in 
the __consumer_offsets partitions.
+ */
+public class StreamsGroup implements Group {
+
+    /**
+     * The protocol type for streams groups. There is only one protocol type, 
"streams".
+     */
+    private static final String PROTOCOL_TYPE = "streams";
+
+    public enum StreamsGroupState {
+        EMPTY("Empty"),
+        NOT_READY("NotReady"),
+        ASSIGNING("Assigning"),
+        RECONCILING("Reconciling"),
+        STABLE("Stable"),
+        DEAD("Dead");
+
+        private final String name;
+
+        private final String lowerCaseName;
+
+        StreamsGroupState(String name) {
+            this.name = name;
+            if (Objects.equals(name, "NotReady")) {
+                this.lowerCaseName = "not_ready";
+            } else {
+                this.lowerCaseName = name.toLowerCase(Locale.ROOT);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return name;
+        }
+
+        public String toLowerCaseString() {
+            return lowerCaseName;
+        }
+    }
+
+    public static class DeadlineAndEpoch {
+
+        static final DeadlineAndEpoch EMPTY = new DeadlineAndEpoch(0L, 0);
+
+        public final long deadlineMs;
+        public final int epoch;
+
+        DeadlineAndEpoch(long deadlineMs, int epoch) {
+            this.deadlineMs = deadlineMs;
+            this.epoch = epoch;
+        }
+    }
+
+    private final LogContext logContext;
+    private final Logger log;
+
+    /**
+     * The snapshot registry.
+     */
+    private final SnapshotRegistry snapshotRegistry;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The group state.
+     */
+    private final TimelineObject<StreamsGroupState> state;
+
+    /**
+     * The group epoch. The epoch is incremented whenever the topology, topic 
metadata or the set of members changes and it will trigger
+     * the computation of a new assignment for the group.
+     */
+    private final TimelineInteger groupEpoch;
+
+    /**
+     * The group members.
+     */
+    private final TimelineHashMap<String, StreamsGroupMember> members;
+
+    /**
+     * The static group members.
+     */
+    private final TimelineHashMap<String, String> staticMembers;
+
+    /**
+     * The metadata associated with each subscribed topic name.
+     */
+    private final TimelineHashMap<String, TopicMetadata> partitionMetadata;
+
+    /**
+     * The target assignment epoch. An assignment epoch smaller than the group 
epoch means that a new assignment is required. The assignment
+     * epoch is updated when a new assignment is installed.
+     */
+    private final TimelineInteger targetAssignmentEpoch;
+
+    /**
+     * The target assignment per member id.
+     */
+    private final TimelineHashMap<String, TasksTuple> targetAssignment;
+
+    /**
+     * Reverse lookup map representing tasks with their current member 
assignments.
+     */
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
invertedTargetActiveTasksAssignment;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
invertedTargetStandbyTasksAssignment;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
invertedTargetWarmupTasksAssignment;
+
+    /**
+     * These maps map each active/standby/warmup task to the process ID(s) of 
their current owner. When a
+     * member revokes a partition, it removes its process ID from this map. 
When a member gets a partition, it adds its process ID to this map.
+     */
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
currentActiveTaskProcessId;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, 
Set<String>>> currentStandbyTaskProcessIds;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, 
Set<String>>> currentWarmupTaskProcessIds;
+
+    /**
+     * The coordinator metrics.
+     */
+    private final GroupCoordinatorMetricsShard metrics;
+
+    /**
+     * The Streams topology.
+     */
+    private final TimelineObject<Optional<StreamsTopology>> topology;
+
+    /**
+     * The configured topology including resolved regular expressions.
+     */
+    private final TimelineObject<Optional<ConfiguredTopology>> 
configuredTopology;
+
+    /**
+     * The metadata refresh deadline. It consists of a timestamp in 
milliseconds together with the group epoch at the time of setting it.
+     * The metadata refresh time is considered as a soft state (read that it 
is not stored in a timeline data structure). It is like this
+     * because it is not persisted to the log. The group epoch is here to 
ensure that the metadata refresh deadline is invalidated if the
+     * group epoch does not correspond to the current group epoch. This can 
happen if the metadata refresh deadline is updated after having
+     * refreshed the metadata but the write operation failed. In this case, 
the time is not automatically rolled back.
+     */
+    private DeadlineAndEpoch metadataRefreshDeadline = DeadlineAndEpoch.EMPTY;
+
+    public StreamsGroup(
+        LogContext logContext,
+        SnapshotRegistry snapshotRegistry,
+        String groupId,
+        GroupCoordinatorMetricsShard metrics
+    ) {
+        this.log = logContext.logger(StreamsGroup.class);
+        this.logContext = logContext;
+        this.snapshotRegistry = Objects.requireNonNull(snapshotRegistry);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = new TimelineObject<>(snapshotRegistry, EMPTY);
+        this.groupEpoch = new TimelineInteger(snapshotRegistry);
+        this.members = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.partitionMetadata = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.targetAssignmentEpoch = new TimelineInteger(snapshotRegistry);
+        this.targetAssignment = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.invertedTargetActiveTasksAssignment = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.invertedTargetStandbyTasksAssignment = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.invertedTargetWarmupTasksAssignment = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.currentActiveTaskProcessId = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.currentStandbyTaskProcessIds = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.currentWarmupTaskProcessIds = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.metrics = Objects.requireNonNull(metrics);
+        this.topology = new TimelineObject<>(snapshotRegistry, 
Optional.empty());
+        this.configuredTopology = new TimelineObject<>(snapshotRegistry, 
Optional.empty());
+    }
+
+    /**
+     * @return The group type (Streams).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.STREAMS;
+    }
+
+    /**
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return state.get().toString();
+    }
+
+    /**
+     * @return The current state as a String with given committedOffset.
+     */
+    public String stateAsString(long committedOffset) {
+        return state.get(committedOffset).toString();
+    }
+
+    /**
+     * @return the group formatted as a list group response based on the 
committed offset.
+     */
+    public ListGroupsResponseData.ListedGroup asListedGroup(long 
committedOffset) {
+        return new ListGroupsResponseData.ListedGroup()
+            .setGroupId(groupId)
+            .setProtocolType(PROTOCOL_TYPE)
+            .setGroupState(state.get(committedOffset).toString())
+            .setGroupType(type().toString());
+    }
+
+    public ConfiguredTopology configuredTopology() {
+        return configuredTopology.get().orElse(null);

Review Comment:
   Done. Note that this is part of the previous PR.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java:
##########
@@ -190,6 +202,55 @@ public GroupCoordinatorMetrics(MetricsRegistry registry, 
Metrics metrics) {
             SHARE_GROUP_COUNT_STATE_TAG, 
ShareGroup.ShareGroupState.DEAD.toString()
         );
 
+        streamsGroupCountMetricName = metrics.metricName(
+            GROUP_COUNT_METRIC_NAME,
+            METRICS_GROUP,
+            "The total number of groups using the streams rebalance protocol.",
+            Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, 
Group.GroupType.STREAMS.toString())
+        );
+
+        streamsGroupCountEmptyMetricName = metrics.metricName(
+            STREAMS_GROUP_COUNT_METRIC_NAME,
+            METRICS_GROUP,
+            "The number of streams groups in empty state.",
+            Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, 
StreamsGroupState.EMPTY.toString())
+        );
+
+        streamsGroupCountAssigningMetricName = metrics.metricName(
+            STREAMS_GROUP_COUNT_METRIC_NAME,
+            METRICS_GROUP,
+            "The number of streams groups in assigning state.",
+            Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, 
StreamsGroupState.ASSIGNING.toString())
+        );
+
+        streamsGroupCountReconcilingMetricName = metrics.metricName(
+            STREAMS_GROUP_COUNT_METRIC_NAME,
+            METRICS_GROUP,
+            "The number of streams groups in reconciling state.",
+            Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, 
StreamsGroupState.RECONCILING.toString())
+        );
+
+        streamsGroupCountStableMetricName = metrics.metricName(
+            STREAMS_GROUP_COUNT_METRIC_NAME,
+            METRICS_GROUP,
+            "The number of streams groups in stable state.",
+            Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, 
StreamsGroupState.STABLE.toString())
+        );
+
+        streamsGroupCountDeadMetricName = metrics.metricName(
+            STREAMS_GROUP_COUNT_METRIC_NAME,
+            METRICS_GROUP,
+            "The number of streams groups in dead state.",
+            Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, 
StreamsGroupState.DEAD.toString())
+        );
+
+        streamsGroupCountNotReadyMetricName = metrics.metricName(
+            STREAMS_GROUP_COUNT_METRIC_NAME,
+            METRICS_GROUP,
+            "The number of streams groups in not ready state.",
+            Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, 
StreamsGroupState.NOT_READY.toString())

Review Comment:
   I don't think individual groups are exposed in the current metrics. There 
are RPCs though to get the state of a specific state `DESCRIBE_GROUP` that can 
be used to implement such monitoring, I suppose.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java:
##########
@@ -0,0 +1,1180 @@
+/*
+ * 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.streams;
+
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.StaleMemberEpochException;
+import org.apache.kafka.common.errors.UnknownMemberIdException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.StreamsGroupDescribeResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.JoinGroupRequest;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
+import org.apache.kafka.coordinator.group.Group;
+import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
+import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
+import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
+import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
+import org.apache.kafka.coordinator.group.streams.topics.InternalTopicManager;
+import org.apache.kafka.image.ClusterImage;
+import org.apache.kafka.image.TopicImage;
+import org.apache.kafka.image.TopicsImage;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.apache.kafka.timeline.TimelineInteger;
+import org.apache.kafka.timeline.TimelineObject;
+
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.ASSIGNING;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.EMPTY;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.NOT_READY;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.RECONCILING;
+import static 
org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState.STABLE;
+
+/**
+ * A Streams Group. All the metadata in this class are backed by records in 
the __consumer_offsets partitions.
+ */
+public class StreamsGroup implements Group {
+
+    /**
+     * The protocol type for streams groups. There is only one protocol type, 
"streams".
+     */
+    private static final String PROTOCOL_TYPE = "streams";
+
+    public enum StreamsGroupState {
+        EMPTY("Empty"),
+        NOT_READY("NotReady"),
+        ASSIGNING("Assigning"),
+        RECONCILING("Reconciling"),
+        STABLE("Stable"),
+        DEAD("Dead");
+
+        private final String name;
+
+        private final String lowerCaseName;
+
+        StreamsGroupState(String name) {
+            this.name = name;
+            if (Objects.equals(name, "NotReady")) {
+                this.lowerCaseName = "not_ready";
+            } else {
+                this.lowerCaseName = name.toLowerCase(Locale.ROOT);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return name;
+        }
+
+        public String toLowerCaseString() {
+            return lowerCaseName;
+        }
+    }
+
+    public static class DeadlineAndEpoch {
+
+        static final DeadlineAndEpoch EMPTY = new DeadlineAndEpoch(0L, 0);
+
+        public final long deadlineMs;
+        public final int epoch;
+
+        DeadlineAndEpoch(long deadlineMs, int epoch) {
+            this.deadlineMs = deadlineMs;
+            this.epoch = epoch;
+        }
+    }
+
+    private final LogContext logContext;
+    private final Logger log;
+
+    /**
+     * The snapshot registry.
+     */
+    private final SnapshotRegistry snapshotRegistry;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The group state.
+     */
+    private final TimelineObject<StreamsGroupState> state;
+
+    /**
+     * The group epoch. The epoch is incremented whenever the topology, topic 
metadata or the set of members changes and it will trigger
+     * the computation of a new assignment for the group.
+     */
+    private final TimelineInteger groupEpoch;
+
+    /**
+     * The group members.
+     */
+    private final TimelineHashMap<String, StreamsGroupMember> members;
+
+    /**
+     * The static group members.
+     */
+    private final TimelineHashMap<String, String> staticMembers;
+
+    /**
+     * The metadata associated with each subscribed topic name.
+     */
+    private final TimelineHashMap<String, TopicMetadata> partitionMetadata;
+
+    /**
+     * The target assignment epoch. An assignment epoch smaller than the group 
epoch means that a new assignment is required. The assignment
+     * epoch is updated when a new assignment is installed.
+     */
+    private final TimelineInteger targetAssignmentEpoch;
+
+    /**
+     * The target assignment per member id.
+     */
+    private final TimelineHashMap<String, TasksTuple> targetAssignment;
+
+    /**
+     * Reverse lookup map representing tasks with their current member 
assignments.
+     */
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
invertedTargetActiveTasksAssignment;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
invertedTargetStandbyTasksAssignment;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
invertedTargetWarmupTasksAssignment;
+
+    /**
+     * These maps map each active/standby/warmup task to the process ID(s) of 
their current owner. When a
+     * member revokes a partition, it removes its process ID from this map. 
When a member gets a partition, it adds its process ID to this map.
+     */
+    private final TimelineHashMap<String, TimelineHashMap<Integer, String>> 
currentActiveTaskProcessId;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, 
Set<String>>> currentStandbyTaskProcessIds;
+    private final TimelineHashMap<String, TimelineHashMap<Integer, 
Set<String>>> currentWarmupTaskProcessIds;
+
+    /**
+     * The coordinator metrics.
+     */
+    private final GroupCoordinatorMetricsShard metrics;
+
+    /**
+     * The Streams topology.
+     */
+    private final TimelineObject<Optional<StreamsTopology>> topology;
+
+    /**
+     * The configured topology including resolved regular expressions.
+     */
+    private final TimelineObject<Optional<ConfiguredTopology>> 
configuredTopology;
+
+    /**
+     * The metadata refresh deadline. It consists of a timestamp in 
milliseconds together with the group epoch at the time of setting it.
+     * The metadata refresh time is considered as a soft state (read that it 
is not stored in a timeline data structure). It is like this
+     * because it is not persisted to the log. The group epoch is here to 
ensure that the metadata refresh deadline is invalidated if the
+     * group epoch does not correspond to the current group epoch. This can 
happen if the metadata refresh deadline is updated after having
+     * refreshed the metadata but the write operation failed. In this case, 
the time is not automatically rolled back.
+     */
+    private DeadlineAndEpoch metadataRefreshDeadline = DeadlineAndEpoch.EMPTY;
+
+    public StreamsGroup(
+        LogContext logContext,
+        SnapshotRegistry snapshotRegistry,
+        String groupId,
+        GroupCoordinatorMetricsShard metrics
+    ) {
+        this.log = logContext.logger(StreamsGroup.class);
+        this.logContext = logContext;
+        this.snapshotRegistry = Objects.requireNonNull(snapshotRegistry);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = new TimelineObject<>(snapshotRegistry, EMPTY);
+        this.groupEpoch = new TimelineInteger(snapshotRegistry);
+        this.members = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.partitionMetadata = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.targetAssignmentEpoch = new TimelineInteger(snapshotRegistry);
+        this.targetAssignment = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.invertedTargetActiveTasksAssignment = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.invertedTargetStandbyTasksAssignment = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.invertedTargetWarmupTasksAssignment = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.currentActiveTaskProcessId = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.currentStandbyTaskProcessIds = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.currentWarmupTaskProcessIds = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.metrics = Objects.requireNonNull(metrics);
+        this.topology = new TimelineObject<>(snapshotRegistry, 
Optional.empty());
+        this.configuredTopology = new TimelineObject<>(snapshotRegistry, 
Optional.empty());
+    }
+
+    /**
+     * @return The group type (Streams).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.STREAMS;
+    }
+
+    /**
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return state.get().toString();
+    }
+
+    /**
+     * @return The current state as a String with given committedOffset.
+     */
+    public String stateAsString(long committedOffset) {
+        return state.get(committedOffset).toString();
+    }
+
+    /**
+     * @return the group formatted as a list group response based on the 
committed offset.
+     */
+    public ListGroupsResponseData.ListedGroup asListedGroup(long 
committedOffset) {
+        return new ListGroupsResponseData.ListedGroup()
+            .setGroupId(groupId)
+            .setProtocolType(PROTOCOL_TYPE)
+            .setGroupState(state.get(committedOffset).toString())
+            .setGroupType(type().toString());
+    }
+
+    public ConfiguredTopology configuredTopology() {
+        return configuredTopology.get().orElse(null);
+    }
+
+    public StreamsTopology topology() {
+        return topology.get().orElse(null);
+    }
+
+    public void setTopology(StreamsTopology topology) {
+        this.topology.set(Optional.of(topology));

Review Comment:
   Done. Note that this is part of the previous PR.



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