You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "jeffkbkim (via GitHub)" <gi...@apache.org> on 2023/05/02 23:36:06 UTC

[GitHub] [kafka] jeffkbkim opened a new pull request, #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

jeffkbkim opened a new pull request, #13663:
URL: https://github.com/apache/kafka/pull/13663

   Rewrites GroupMetadata as GenericGroup that will be used with the new group coordinator.
   
   Written on top of https://github.com/apache/kafka/pull/13644, will rebase once it's merged. these files under `/generic` should be reviewed:
   * `GenericGroup.java`
   * `GenericGroup.java` // TODO
   * `CommitRecordMetadataAndOffset.java`
   * `GenericGroupState.java`
   * `OffsetAndMetadata.java`
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] clolov commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1192091604


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;

Review Comment:
   Sorry, I do not understand. This field is currently private so the only way we can set it now is either in the constructor or via a setter. Do you mean that in the near future either we will make this field public or we will add a setter which will then be called by the GroupMetadataManager to possibly set this to an empty Optional? If so, then okay, this makes sense.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190358529


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all member's rebalance timeout.
+     */
+    public int rebalanceTimeoutMs() {
+        int maxRebalanceTimeoutMs = 0;
+        for (GenericGroupMember member : members.values()) {
+            maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
+        }
+        return maxRebalanceTimeoutMs;
+    }
+
+    /**
+     * Generate a member id from the given client and group instance ids.
+     *
+     * @param clientId the client id.
+     * @param groupInstanceId the group instance id.
+     * @return the generated id.
+     */

Review Comment:
   Should we indent this better? 
        ```
        * @param clientId                the client id.
        * @param groupInstanceId the group instance id.
        * @return                              the generated id.
   ```
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1189924161


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {

Review Comment:
   do we need to add javadoc for overloaded methods? i wrote one for `add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future)`



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191161586


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *

Review Comment:
   nit: This empty line can be removed.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }

Review Comment:
   Was this method in the old implementation? The name is a bit weird because a generic group could use different protocol types (e.g. connect). Should it be named `useConsumerGroupProtocol`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {

Review Comment:
   nit: Should we use `isInState` or `inState` because `isState` reads weird.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     * @param future  the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, choose a
+     * new leader from one of the joined members.
+     *
+     * Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",

Review Comment:
   nit: The indentation of these lines it not consistent with the previous `log.info` at L409. Let's try to use the same format.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("range");
+        expectedProtocols.add("bar");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("foo");
+        expectedProtocols.add("bar");
+        assertFalse(group.supportsProtocols(protocolType, expectedProtocols));
+    }
+
+    @Test
+    public void testSubscribedTopics() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    ConsumerProtocol.serializeSubscription(
+                        new ConsumerPartitionAssignor.Subscription(
+                            Collections.singletonList("foo")
+                        )
+                    ).array()
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member);
+
+        group.initNextGeneration();
+
+        Set<String> expectedTopics = new HashSet<>(Collections.singleton("foo"));
+        assertEquals(expectedTopics, group.subscribedTopics().get());
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.remove(memberId);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics());
+
+        GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithFaultyProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testSubscribedTopicsNonConsumerGroup() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            "My Protocol",
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithNonConsumerProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testInitNextGeneration() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member, new CompletableFuture<>());
+
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertEquals("roundrobin", group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testInitNextGenerationEmptyGroup() {
+        assertEquals(EMPTY, group.currentState());
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        List<Protocol> newProtocols = Arrays.asList(
+            new Protocol(
+                "range",
+                new byte[0]
+            ),
+            new Protocol(
+                "roundrobin",
+                new byte[0]
+            )
+        );
+        int newRebalanceTimeoutMs = 120000;
+        int newSessionTimeoutMs = 20000;
+        group.updateMember(member, newProtocols, newRebalanceTimeoutMs, newSessionTimeoutMs, null);
+
+        assertEquals(group.rebalanceTimeoutMs(), newRebalanceTimeoutMs);
+        assertEquals(member.sessionTimeoutMs(), newSessionTimeoutMs);
+        assertEquals(newProtocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testReplaceGroupInstanceWithNonExistingMember() {
+        String newMemberId = "newMemberId";
+        assertThrows(IllegalArgumentException.class, () ->
+            group.replaceStaticMember(groupInstanceId, memberId, newMemberId));
+    }
+
+    @Test
+    public void testReplaceGroupInstance() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.of(groupInstanceId),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean joinAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((joinGroupResult, __) ->
+            joinAwaitingMemberFenced.set(joinGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));

Review Comment:
   nit: This is a bit weird. Why can't we verify the future after `replaceStaticMember` is called?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("range");
+        expectedProtocols.add("bar");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("foo");
+        expectedProtocols.add("bar");
+        assertFalse(group.supportsProtocols(protocolType, expectedProtocols));
+    }
+
+    @Test
+    public void testSubscribedTopics() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    ConsumerProtocol.serializeSubscription(
+                        new ConsumerPartitionAssignor.Subscription(
+                            Collections.singletonList("foo")
+                        )
+                    ).array()
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member);
+
+        group.initNextGeneration();
+
+        Set<String> expectedTopics = new HashSet<>(Collections.singleton("foo"));
+        assertEquals(expectedTopics, group.subscribedTopics().get());
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.remove(memberId);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics());
+
+        GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithFaultyProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testSubscribedTopicsNonConsumerGroup() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            "My Protocol",
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithNonConsumerProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testInitNextGeneration() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member, new CompletableFuture<>());
+
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertEquals("roundrobin", group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testInitNextGenerationEmptyGroup() {
+        assertEquals(EMPTY, group.currentState());
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        List<Protocol> newProtocols = Arrays.asList(
+            new Protocol(
+                "range",
+                new byte[0]
+            ),
+            new Protocol(
+                "roundrobin",
+                new byte[0]
+            )
+        );
+        int newRebalanceTimeoutMs = 120000;
+        int newSessionTimeoutMs = 20000;
+        group.updateMember(member, newProtocols, newRebalanceTimeoutMs, newSessionTimeoutMs, null);
+
+        assertEquals(group.rebalanceTimeoutMs(), newRebalanceTimeoutMs);
+        assertEquals(member.sessionTimeoutMs(), newSessionTimeoutMs);
+        assertEquals(newProtocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testReplaceGroupInstanceWithNonExistingMember() {
+        String newMemberId = "newMemberId";
+        assertThrows(IllegalArgumentException.class, () ->
+            group.replaceStaticMember(groupInstanceId, memberId, newMemberId));
+    }
+
+    @Test
+    public void testReplaceGroupInstance() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.of(groupInstanceId),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean joinAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((joinGroupResult, __) ->
+            joinAwaitingMemberFenced.set(joinGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        group.add(member, joinGroupFuture);
+
+        AtomicBoolean syncAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<SyncGroupResponseData> syncGroupFuture = new CompletableFuture<>();
+        syncGroupFuture.whenComplete((syncGroupResult, __) ->
+            syncAwaitingMemberFenced.set(syncGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        member.setAwaitingSyncFuture(syncGroupFuture);
+
+        assertTrue(group.isLeader(memberId));
+        assertEquals(memberId, group.staticMemberId(groupInstanceId));
+
+        String newMemberId = "newMemberId";
+        group.replaceStaticMember(groupInstanceId, memberId, newMemberId);
+        assertTrue(group.isLeader(newMemberId));
+        assertEquals(newMemberId, group.staticMemberId(groupInstanceId));
+        assertTrue(joinAwaitingMemberFenced.get());
+        assertTrue(syncAwaitingMemberFenced.get());
+        assertFalse(member.isAwaitingJoin());
+        assertFalse(member.isAwaitingSync());
+    }
+
+    @Test
+    public void testCompleteJoinFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean invoked = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((__, ___) ->
+            invoked.set(true));
+        group.add(member, joinGroupFuture);
+
+        assertTrue(group.hasAllMembersJoined());
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(member.memberId())
+            .setErrorCode(Errors.NONE.code()));
+
+        assertTrue(invoked.get());
+        assertFalse(member.isAwaitingJoin());
+    }
+
+    @Test
+    public void testNotCompleteJoinFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        assertFalse(member.isAwaitingJoin());
+        group.completeJoinFuture(member, new JoinGroupResponseData()

Review Comment:
   nit: Should we change `completeJoinFuture` to return a boolean? This test is a bit useless otherwise.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));

Review Comment:
   nit: We could use `mkSet` to simply the code.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;

Review Comment:
   nit: Do we need to keep a reference to the context? I suppose that it is not required.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     * @param future  the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, choose a
+     * new leader from one of the joined members.
+     *
+     * Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;

Review Comment:
   It seems that `true` is returned in this case in the scala code, no? 



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     * @param future  the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {

Review Comment:
   nit: Should we add an overload to `supportsProtocols` which takes a member? That would simplify this condition here and make it more readable.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     * @param future  the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, choose a
+     * new leader from one of the joined members.
+     *
+     * Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId  the group instance id.
+     * @param oldMemberId      the old member id.
+     * @param newMemberId      the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {

Review Comment:
   nit: `numAwaitingJoinResponse`?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+

Review Comment:
   nit: We can remove this empty line.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("range");
+        expectedProtocols.add("bar");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("foo");
+        expectedProtocols.add("bar");
+        assertFalse(group.supportsProtocols(protocolType, expectedProtocols));
+    }
+
+    @Test
+    public void testSubscribedTopics() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    ConsumerProtocol.serializeSubscription(
+                        new ConsumerPartitionAssignor.Subscription(
+                            Collections.singletonList("foo")
+                        )
+                    ).array()
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member);
+
+        group.initNextGeneration();
+
+        Set<String> expectedTopics = new HashSet<>(Collections.singleton("foo"));
+        assertEquals(expectedTopics, group.subscribedTopics().get());
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.remove(memberId);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics());
+
+        GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithFaultyProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testSubscribedTopicsNonConsumerGroup() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            "My Protocol",
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithNonConsumerProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testInitNextGeneration() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member, new CompletableFuture<>());
+
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertEquals("roundrobin", group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testInitNextGenerationEmptyGroup() {
+        assertEquals(EMPTY, group.currentState());
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        List<Protocol> newProtocols = Arrays.asList(
+            new Protocol(
+                "range",
+                new byte[0]
+            ),
+            new Protocol(
+                "roundrobin",
+                new byte[0]
+            )
+        );
+        int newRebalanceTimeoutMs = 120000;
+        int newSessionTimeoutMs = 20000;
+        group.updateMember(member, newProtocols, newRebalanceTimeoutMs, newSessionTimeoutMs, null);
+
+        assertEquals(group.rebalanceTimeoutMs(), newRebalanceTimeoutMs);
+        assertEquals(member.sessionTimeoutMs(), newSessionTimeoutMs);
+        assertEquals(newProtocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testReplaceGroupInstanceWithNonExistingMember() {
+        String newMemberId = "newMemberId";
+        assertThrows(IllegalArgumentException.class, () ->
+            group.replaceStaticMember(groupInstanceId, memberId, newMemberId));
+    }
+
+    @Test
+    public void testReplaceGroupInstance() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.of(groupInstanceId),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean joinAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((joinGroupResult, __) ->
+            joinAwaitingMemberFenced.set(joinGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        group.add(member, joinGroupFuture);
+
+        AtomicBoolean syncAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<SyncGroupResponseData> syncGroupFuture = new CompletableFuture<>();
+        syncGroupFuture.whenComplete((syncGroupResult, __) ->
+            syncAwaitingMemberFenced.set(syncGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));

Review Comment:
   ditto.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("range");
+        expectedProtocols.add("bar");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("foo");
+        expectedProtocols.add("bar");
+        assertFalse(group.supportsProtocols(protocolType, expectedProtocols));
+    }
+
+    @Test
+    public void testSubscribedTopics() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    ConsumerProtocol.serializeSubscription(
+                        new ConsumerPartitionAssignor.Subscription(
+                            Collections.singletonList("foo")
+                        )
+                    ).array()
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member);
+
+        group.initNextGeneration();
+
+        Set<String> expectedTopics = new HashSet<>(Collections.singleton("foo"));
+        assertEquals(expectedTopics, group.subscribedTopics().get());
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.remove(memberId);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics());
+
+        GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithFaultyProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testSubscribedTopicsNonConsumerGroup() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            "My Protocol",
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithNonConsumerProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testInitNextGeneration() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member, new CompletableFuture<>());
+
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertEquals("roundrobin", group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testInitNextGenerationEmptyGroup() {
+        assertEquals(EMPTY, group.currentState());
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        List<Protocol> newProtocols = Arrays.asList(
+            new Protocol(
+                "range",
+                new byte[0]
+            ),
+            new Protocol(
+                "roundrobin",
+                new byte[0]
+            )
+        );
+        int newRebalanceTimeoutMs = 120000;
+        int newSessionTimeoutMs = 20000;
+        group.updateMember(member, newProtocols, newRebalanceTimeoutMs, newSessionTimeoutMs, null);
+
+        assertEquals(group.rebalanceTimeoutMs(), newRebalanceTimeoutMs);
+        assertEquals(member.sessionTimeoutMs(), newSessionTimeoutMs);
+        assertEquals(newProtocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testReplaceGroupInstanceWithNonExistingMember() {
+        String newMemberId = "newMemberId";
+        assertThrows(IllegalArgumentException.class, () ->
+            group.replaceStaticMember(groupInstanceId, memberId, newMemberId));
+    }
+
+    @Test
+    public void testReplaceGroupInstance() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.of(groupInstanceId),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean joinAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((joinGroupResult, __) ->
+            joinAwaitingMemberFenced.set(joinGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        group.add(member, joinGroupFuture);
+
+        AtomicBoolean syncAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<SyncGroupResponseData> syncGroupFuture = new CompletableFuture<>();
+        syncGroupFuture.whenComplete((syncGroupResult, __) ->
+            syncAwaitingMemberFenced.set(syncGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        member.setAwaitingSyncFuture(syncGroupFuture);
+
+        assertTrue(group.isLeader(memberId));
+        assertEquals(memberId, group.staticMemberId(groupInstanceId));
+
+        String newMemberId = "newMemberId";
+        group.replaceStaticMember(groupInstanceId, memberId, newMemberId);
+        assertTrue(group.isLeader(newMemberId));
+        assertEquals(newMemberId, group.staticMemberId(groupInstanceId));
+        assertTrue(joinAwaitingMemberFenced.get());
+        assertTrue(syncAwaitingMemberFenced.get());
+        assertFalse(member.isAwaitingJoin());
+        assertFalse(member.isAwaitingSync());
+    }
+
+    @Test
+    public void testCompleteJoinFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean invoked = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((__, ___) ->
+            invoked.set(true));

Review Comment:
   ditto.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("range");
+        expectedProtocols.add("bar");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("foo");
+        expectedProtocols.add("bar");
+        assertFalse(group.supportsProtocols(protocolType, expectedProtocols));
+    }
+
+    @Test
+    public void testSubscribedTopics() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    ConsumerProtocol.serializeSubscription(
+                        new ConsumerPartitionAssignor.Subscription(
+                            Collections.singletonList("foo")
+                        )
+                    ).array()
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member);
+
+        group.initNextGeneration();
+
+        Set<String> expectedTopics = new HashSet<>(Collections.singleton("foo"));
+        assertEquals(expectedTopics, group.subscribedTopics().get());
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.remove(memberId);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics());
+
+        GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithFaultyProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testSubscribedTopicsNonConsumerGroup() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            "My Protocol",
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithNonConsumerProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testInitNextGeneration() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member, new CompletableFuture<>());
+
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertEquals("roundrobin", group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testInitNextGenerationEmptyGroup() {
+        assertEquals(EMPTY, group.currentState());
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        List<Protocol> newProtocols = Arrays.asList(
+            new Protocol(
+                "range",
+                new byte[0]
+            ),
+            new Protocol(
+                "roundrobin",
+                new byte[0]
+            )
+        );
+        int newRebalanceTimeoutMs = 120000;
+        int newSessionTimeoutMs = 20000;
+        group.updateMember(member, newProtocols, newRebalanceTimeoutMs, newSessionTimeoutMs, null);
+
+        assertEquals(group.rebalanceTimeoutMs(), newRebalanceTimeoutMs);
+        assertEquals(member.sessionTimeoutMs(), newSessionTimeoutMs);
+        assertEquals(newProtocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testReplaceGroupInstanceWithNonExistingMember() {
+        String newMemberId = "newMemberId";
+        assertThrows(IllegalArgumentException.class, () ->
+            group.replaceStaticMember(groupInstanceId, memberId, newMemberId));
+    }
+
+    @Test
+    public void testReplaceGroupInstance() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.of(groupInstanceId),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean joinAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((joinGroupResult, __) ->
+            joinAwaitingMemberFenced.set(joinGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        group.add(member, joinGroupFuture);
+
+        AtomicBoolean syncAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<SyncGroupResponseData> syncGroupFuture = new CompletableFuture<>();
+        syncGroupFuture.whenComplete((syncGroupResult, __) ->
+            syncAwaitingMemberFenced.set(syncGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        member.setAwaitingSyncFuture(syncGroupFuture);
+
+        assertTrue(group.isLeader(memberId));
+        assertEquals(memberId, group.staticMemberId(groupInstanceId));
+
+        String newMemberId = "newMemberId";
+        group.replaceStaticMember(groupInstanceId, memberId, newMemberId);
+        assertTrue(group.isLeader(newMemberId));
+        assertEquals(newMemberId, group.staticMemberId(groupInstanceId));
+        assertTrue(joinAwaitingMemberFenced.get());
+        assertTrue(syncAwaitingMemberFenced.get());
+        assertFalse(member.isAwaitingJoin());
+        assertFalse(member.isAwaitingSync());
+    }
+
+    @Test
+    public void testCompleteJoinFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean invoked = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((__, ___) ->
+            invoked.set(true));
+        group.add(member, joinGroupFuture);
+
+        assertTrue(group.hasAllMembersJoined());
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(member.memberId())
+            .setErrorCode(Errors.NONE.code()));
+
+        assertTrue(invoked.get());
+        assertFalse(member.isAwaitingJoin());
+    }
+
+    @Test
+    public void testNotCompleteJoinFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        assertFalse(member.isAwaitingJoin());
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(member.memberId())
+            .setErrorCode(Errors.NONE.code()));
+
+        assertFalse(member.isAwaitingJoin());
+    }
+
+    @Test
+    public void testCompleteSyncFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+        CompletableFuture<SyncGroupResponseData> syncGroupFuture = new CompletableFuture<>();
+        member.setAwaitingSyncFuture(syncGroupFuture);
+
+        assertTrue(group.completeSyncFuture(member, new SyncGroupResponseData()
+            .setErrorCode(Errors.NONE.code())));
+
+        assertFalse(member.isAwaitingSync());
+    }
+
+    @Test
+    public void testNotCompleteSyncFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        assertFalse(group.completeSyncFuture(member, new SyncGroupResponseData()
+            .setErrorCode(Errors.NONE.code())));
+
+        assertFalse(member.isAwaitingSync());

Review Comment:
   Should we assert this before calling `completeSyncFuture`?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("range");
+        expectedProtocols.add("bar");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("foo");
+        expectedProtocols.add("bar");
+        assertFalse(group.supportsProtocols(protocolType, expectedProtocols));
+    }
+
+    @Test
+    public void testSubscribedTopics() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    ConsumerProtocol.serializeSubscription(
+                        new ConsumerPartitionAssignor.Subscription(
+                            Collections.singletonList("foo")
+                        )
+                    ).array()
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member);
+
+        group.initNextGeneration();
+
+        Set<String> expectedTopics = new HashSet<>(Collections.singleton("foo"));
+        assertEquals(expectedTopics, group.subscribedTopics().get());
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.remove(memberId);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics());
+
+        GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithFaultyProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testSubscribedTopicsNonConsumerGroup() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            "My Protocol",
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithNonConsumerProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testInitNextGeneration() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member, new CompletableFuture<>());
+
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertEquals("roundrobin", group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testInitNextGenerationEmptyGroup() {
+        assertEquals(EMPTY, group.currentState());
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        List<Protocol> newProtocols = Arrays.asList(
+            new Protocol(
+                "range",
+                new byte[0]
+            ),
+            new Protocol(
+                "roundrobin",
+                new byte[0]
+            )
+        );
+        int newRebalanceTimeoutMs = 120000;
+        int newSessionTimeoutMs = 20000;
+        group.updateMember(member, newProtocols, newRebalanceTimeoutMs, newSessionTimeoutMs, null);
+
+        assertEquals(group.rebalanceTimeoutMs(), newRebalanceTimeoutMs);
+        assertEquals(member.sessionTimeoutMs(), newSessionTimeoutMs);
+        assertEquals(newProtocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testReplaceGroupInstanceWithNonExistingMember() {
+        String newMemberId = "newMemberId";
+        assertThrows(IllegalArgumentException.class, () ->
+            group.replaceStaticMember(groupInstanceId, memberId, newMemberId));
+    }
+
+    @Test
+    public void testReplaceGroupInstance() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.of(groupInstanceId),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean joinAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((joinGroupResult, __) ->
+            joinAwaitingMemberFenced.set(joinGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        group.add(member, joinGroupFuture);
+
+        AtomicBoolean syncAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<SyncGroupResponseData> syncGroupFuture = new CompletableFuture<>();
+        syncGroupFuture.whenComplete((syncGroupResult, __) ->
+            syncAwaitingMemberFenced.set(syncGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));
+        member.setAwaitingSyncFuture(syncGroupFuture);
+
+        assertTrue(group.isLeader(memberId));
+        assertEquals(memberId, group.staticMemberId(groupInstanceId));
+
+        String newMemberId = "newMemberId";
+        group.replaceStaticMember(groupInstanceId, memberId, newMemberId);
+        assertTrue(group.isLeader(newMemberId));
+        assertEquals(newMemberId, group.staticMemberId(groupInstanceId));
+        assertTrue(joinAwaitingMemberFenced.get());
+        assertTrue(syncAwaitingMemberFenced.get());
+        assertFalse(member.isAwaitingJoin());
+        assertFalse(member.isAwaitingSync());
+    }
+
+    @Test
+    public void testCompleteJoinFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean invoked = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((__, ___) ->
+            invoked.set(true));
+        group.add(member, joinGroupFuture);
+
+        assertTrue(group.hasAllMembersJoined());
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(member.memberId())
+            .setErrorCode(Errors.NONE.code()));
+
+        assertTrue(invoked.get());
+        assertFalse(member.isAwaitingJoin());
+    }
+
+    @Test
+    public void testNotCompleteJoinFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        assertFalse(member.isAwaitingJoin());
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(member.memberId())
+            .setErrorCode(Errors.NONE.code()));
+
+        assertFalse(member.isAwaitingJoin());
+    }
+
+    @Test
+    public void testCompleteSyncFuture() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+        CompletableFuture<SyncGroupResponseData> syncGroupFuture = new CompletableFuture<>();
+        member.setAwaitingSyncFuture(syncGroupFuture);
+
+        assertTrue(group.completeSyncFuture(member, new SyncGroupResponseData()
+            .setErrorCode(Errors.NONE.code())));
+
+        assertFalse(member.isAwaitingSync());

Review Comment:
   Should we assert `syncGroupFuture`?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190295837


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.

Review Comment:
   would it help to also mention what a generic group is here?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1188772104


##########
core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala:
##########
@@ -160,6 +160,7 @@ class GroupMetadataTest {
     assertThrows(classOf[IllegalStateException], () => group.transitionTo(CompletingRebalance))
   }
 
+  @Test

Review Comment:
   created https://github.com/apache/kafka/pull/13694



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190358529


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all member's rebalance timeout.
+     */
+    public int rebalanceTimeoutMs() {
+        int maxRebalanceTimeoutMs = 0;
+        for (GenericGroupMember member : members.values()) {
+            maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
+        }
+        return maxRebalanceTimeoutMs;
+    }
+
+    /**
+     * Generate a member id from the given client and group instance ids.
+     *
+     * @param clientId the client id.
+     * @param groupInstanceId the group instance id.
+     * @return the generated id.
+     */

Review Comment:
   Should we indent this better? 
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac merged pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac merged PR #13663:
URL: https://github.com/apache/kafka/pull/13663


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190308136


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.

Review Comment:
   nit: Get the member metadata associated* with the provided member id.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190177045


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {

Review Comment:
   will change to hasMemberId



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190313449


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,936 @@
+/*
+ * 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.
+ */
+

Review Comment:
   nit: remove extra line



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190358529


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all member's rebalance timeout.
+     */
+    public int rebalanceTimeoutMs() {
+        int maxRebalanceTimeoutMs = 0;
+        for (GenericGroupMember member : members.values()) {
+            maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
+        }
+        return maxRebalanceTimeoutMs;
+    }
+
+    /**
+     * Generate a member id from the given client and group instance ids.
+     *
+     * @param clientId the client id.
+     * @param groupInstanceId the group instance id.
+     * @return the generated id.
+     */

Review Comment:
   Should we indent this better? 
   ```
        * @param clientId                the client id.
        * @param groupInstanceId the group instance id.
        * @return                              the generated id.
   ```
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191359030


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("range");
+        expectedProtocols.add("bar");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        expectedProtocols.clear();
+        expectedProtocols.add("foo");
+        expectedProtocols.add("bar");
+        assertFalse(group.supportsProtocols(protocolType, expectedProtocols));
+    }
+
+    @Test
+    public void testSubscribedTopics() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    ConsumerProtocol.serializeSubscription(
+                        new ConsumerPartitionAssignor.Subscription(
+                            Collections.singletonList("foo")
+                        )
+                    ).array()
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member);
+
+        group.initNextGeneration();
+
+        Set<String> expectedTopics = new HashSet<>(Collections.singleton("foo"));
+        assertEquals(expectedTopics, group.subscribedTopics().get());
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.remove(memberId);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics());
+
+        GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithFaultyProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testSubscribedTopicsNonConsumerGroup() {
+        // not able to compute it for a newly created group
+        assertEquals(Optional.empty(), group.subscribedTopics());
+
+        GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            "My Protocol",
+            Collections.singletonList(
+                new Protocol(
+                    "range",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(memberWithNonConsumerProtocol);
+
+        group.initNextGeneration();
+
+        assertEquals(Optional.empty(), group.subscribedTopics());
+    }
+
+    @Test
+    public void testInitNextGeneration() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.add(member, new CompletableFuture<>());
+
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertEquals("roundrobin", group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testInitNextGenerationEmptyGroup() {
+        assertEquals(EMPTY, group.currentState());
+        assertEquals(0, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.initNextGeneration();
+
+        assertEquals(1, group.generationId());
+        assertNull(group.protocolName().orElse(null));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        group.add(member);
+
+        List<Protocol> newProtocols = Arrays.asList(
+            new Protocol(
+                "range",
+                new byte[0]
+            ),
+            new Protocol(
+                "roundrobin",
+                new byte[0]
+            )
+        );
+        int newRebalanceTimeoutMs = 120000;
+        int newSessionTimeoutMs = 20000;
+        group.updateMember(member, newProtocols, newRebalanceTimeoutMs, newSessionTimeoutMs, null);
+
+        assertEquals(group.rebalanceTimeoutMs(), newRebalanceTimeoutMs);
+        assertEquals(member.sessionTimeoutMs(), newSessionTimeoutMs);
+        assertEquals(newProtocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testReplaceGroupInstanceWithNonExistingMember() {
+        String newMemberId = "newMemberId";
+        assertThrows(IllegalArgumentException.class, () ->
+            group.replaceStaticMember(groupInstanceId, memberId, newMemberId));
+    }
+
+    @Test
+    public void testReplaceGroupInstance() {
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            Optional.of(groupInstanceId),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            Collections.singletonList(
+                new Protocol(
+                    "roundrobin",
+                    new byte[0]
+                )
+            )
+        );
+
+        AtomicBoolean joinAwaitingMemberFenced = new AtomicBoolean(false);
+        CompletableFuture<JoinGroupResponseData> joinGroupFuture = new CompletableFuture<>();
+        joinGroupFuture.whenComplete((joinGroupResult, __) ->
+            joinAwaitingMemberFenced.set(joinGroupResult.errorCode() == Errors.FENCED_INSTANCE_ID.code()));

Review Comment:
   thanks. updated



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] clolov commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191430711


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;

Review Comment:
   Is this necessary to be an Optional? As far as I see we immediately define it in the constructor and we never set it equal to something which is empty. Am I missing something obvious?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191381128


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,1012 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isInState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean usesConsumerGroupProtocol() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     */
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     * @param future  the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(member)) {
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, choose a
+     * new leader from one of the joined members.
+     *
+     * Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return true if a new leader was elected or the existing
+     *         leader rejoined, false otherwise.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                                "failed to join before the rebalance timeout. Member {} " +
+                                "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId  the group instance id.
+     * @param oldMemberId      the old member id.
+     * @param newMemberId      the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaitingJoinResponse() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all members' rebalance timeout.
+     */
+    public int rebalanceTimeoutMs() {
+        int maxRebalanceTimeoutMs = 0;
+        for (GenericGroupMember member : members.values()) {
+            maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
+        }
+        return maxRebalanceTimeoutMs;
+    }
+
+    /**
+     * Generate a member id from the given client and group instance ids.
+     *
+     * @param clientId         the client id.
+     * @param groupInstanceId  the group instance id.
+     * @return the generated id.
+     */
+    public String generateMemberId(String clientId, Optional<String> groupInstanceId) {
+        return groupInstanceId.map(s -> s + MEMBER_ID_DELIMITER + UUID.randomUUID())
+            .orElseGet(() -> clientId + MEMBER_ID_DELIMITER + UUID.randomUUID());
+    }
+
+    /**
+     * Verify the member id is up to date for static members. Return true if both conditions met:
+     *   1. given member is a known static member to group
+     *   2. group stored member id doesn't match with given member id
+     *
+     * @param groupInstanceId  the group instance id.
+     * @param memberId         the member id.
+     * @return whether the static member is fenced based on the condition above.
+     */
+    public boolean isStaticMemberFenced(
+        String groupInstanceId,
+        String memberId
+    ) {
+        String existingMemberId = staticMemberId(groupInstanceId);
+        return existingMemberId != null && !existingMemberId.equals(memberId);
+    }
+
+    /**
+     * @return whether the group can rebalance.
+     */
+    public boolean canRebalance() {
+        return PREPARING_REBALANCE.validPreviousStates().contains(state);
+    }
+
+    /**
+     * Transition to a group state.
+     * @param groupState the group state.
+     */
+    public void transitionTo(GenericGroupState groupState) {
+        assertValidTransition(groupState);
+        state = groupState;
+        currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * Select a protocol that will be used for this group. Each member
+     * will vote for a protocol and the one with the most votes will
+     * be selected. Only a protocol that is supported by all members
+     * can be selected.
+     *
+     * @return the name of the selected protocol.
+     */
+    public String selectProtocol() {
+        if (members.isEmpty()) {
+            throw new IllegalStateException("Cannot select protocol for empty group");
+        }
+
+        // select the protocol for this group which is supported by all members
+        Set<String> candidates = candidateProtocols();
+
+        // let each member vote for one of the protocols
+        Map<String, Integer> votesByProtocol = new HashMap<>();
+        allMembers().stream().map(member -> member.vote(candidates))
+            .forEach(protocolName -> {
+                int count = votesByProtocol.getOrDefault(protocolName, 0);
+                votesByProtocol.put(protocolName, count + 1);
+            });
+
+        // choose the one with the most votes
+        return votesByProtocol.entrySet().stream()
+            .max(Comparator.comparingInt(Map.Entry::getValue))
+            .map(Map.Entry::getKey).orElse(null);
+    }
+
+    /**
+     * Increment the protocol count for all of the member's
+     * supported protocols.
+     *
+     * @param member the member.
+     */
+    private void incrementSupportedProtocols(GenericGroupMember member) {
+        member.supportedProtocols().forEach(protocol -> {
+            int count = supportedProtocols.getOrDefault(protocol.name(), 0);
+            supportedProtocols.put(protocol.name(), count + 1);
+        });
+    }
+
+    /**
+     * Decrement the protocol count for all of the member's
+     * supported protocols.
+     *
+     * @param member the member.
+     */
+    private void decrementSupportedProtocols(GenericGroupMember member) {
+        member.supportedProtocols().forEach(protocol -> {
+            int count = supportedProtocols.getOrDefault(protocol.name(), 0);
+            supportedProtocols.put(protocol.name(), count - 1);
+        });
+    }
+
+    /**
+     * A candidate protocol must be supported by all members.
+     *
+     * @return a set of candidate protocols that can be chosen as the protocol
+     *         for the group.
+     */
+    private Set<String> candidateProtocols() {
+        // get the set of protocols that are commonly supported by all members
+        return supportedProtocols.entrySet().stream()
+            .filter(protocol -> protocol.getValue() == members.size())
+            .map(Map.Entry::getKey).collect(Collectors.toSet());
+    }
+
+    /**
+     * Checks whether at least one of the given protocols can be supported. A
+     * protocol can be supported if it is supported by all members.
+     *
+     * @param member               the member to check.
+     * @return a boolean based on the condition mentioned above.
+     */
+    public boolean supportsProtocols(GenericGroupMember member) {
+        return supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols())
+        );
+    }
+
+    /**
+     * Checks whether at least one of the given protocols can be supported. A
+     * protocol can be supported if it is supported by all members.
+     *
+     * @param memberProtocolType  the member protocol type.
+     * @param memberProtocols     the set of protocol names.
+     * @return a boolean based on the condition mentioned above.
+     */
+    public boolean supportsProtocols(String memberProtocolType, Set<String> memberProtocols) {
+        if (isInState(EMPTY)) {
+            return !memberProtocolType.isEmpty() && !memberProtocols.isEmpty();
+        } else {
+            return protocolType.map(type -> type.equals(memberProtocolType)).orElse(false) &&
+                memberProtocols.stream()
+                    .anyMatch(name -> supportedProtocols.getOrDefault(name, 0) == members.size());
+        }
+    }
+
+    /**
+     * @return the topics that the group is subscribed to.
+     */
+    public Optional<Set<String>> subscribedTopics() {
+        return subscribedTopics;
+    }
+
+    /**
+     * Returns true if the consumer group is actively subscribed to the topic. When the consumer
+     * group does not know, because the information is not available yet or because the it has
+     * failed to parse the Consumer Protocol, it returns true to be safe.
+     *
+     * @param topic the topic name.
+     * @return whether the group is subscribed to the topic.
+     */
+    public boolean isSubscribedToTopic(String topic) {
+        return subscribedTopics.map(topics -> topics.contains(topic))
+            .orElse(true);
+    }
+
+    /**
+     * Collects the set of topics that the members are subscribed to when the Protocol Type is equal
+     * to 'consumer'. None is returned if
+     * - the protocol type is not equal to 'consumer';
+     * - the protocol is not defined yet; or
+     * - the protocol metadata does not comply with the schema.
+     *
+     * @return the subscribed topics or None based on the condition above.
+     */
+    Optional<Set<String>> computeSubscribedTopics() {
+        if (!protocolType.isPresent()) {
+            return Optional.empty();
+        }
+        String type = protocolType.get();
+        if (!type.equals(ConsumerProtocol.PROTOCOL_TYPE)) {
+            return Optional.empty();
+        }
+        if (members.isEmpty()) {
+            return Optional.of(Collections.emptySet());
+        }
+
+        if (protocolName.isPresent()) {
+            try {
+                Set<String> allSubscribedTopics = new HashSet<>();
+                members.values().forEach(member -> {
+                    ByteBuffer buffer = ByteBuffer.wrap(member.metadata(protocolName.get()));
+                    ConsumerProtocol.deserializeVersion(buffer);
+                    allSubscribedTopics.addAll(new HashSet<>(
+                        ConsumerProtocol.deserializeSubscription(buffer, (short) 0).topics()
+                    ));
+                });
+                return Optional.of(allSubscribedTopics);
+            } catch (SchemaException e) {
+                log.warn("Failed to parse Consumer Protocol " + ConsumerProtocol.PROTOCOL_TYPE + ":" +
+                    protocolName.get() + " of group " + groupId + ". " +
+                    "Consumer group coordinator is not aware of the subscribed topics.", e);
+            }
+        }
+
+        return Optional.empty();
+    }
+
+    /**
+     * Update a member.
+     *
+     * @param member              the member.
+     * @param protocols           the list of protocols.
+     * @param rebalanceTimeoutMs  the rebalance timeout in milliseconds.
+     * @param sessionTimeoutMs    the session timeout in milliseconds.
+     * @param future              the future that is invoked once the join phase is complete.
+     */
+    public void updateMember(
+        GenericGroupMember member,
+        List<Protocol> protocols,
+        int rebalanceTimeoutMs,
+        int sessionTimeoutMs,
+        CompletableFuture<JoinGroupResponseData> future
+    ) {
+        decrementSupportedProtocols(member);
+        member.setSupportedProtocols(protocols);
+        incrementSupportedProtocols(member);
+        member.setRebalanceTimeoutMs(rebalanceTimeoutMs);
+        member.setSessionTimeoutMs(sessionTimeoutMs);
+
+        if (future != null && !member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        } else if (future == null && member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse--;
+        }
+        member.setAwaitingJoinFuture(future);
+    }
+
+    /**
+     * Complete the join future.
+     * 
+     * @param member    the member.
+     * @param response  the join response to complete the future with.
+     * @return true if a join future actually completes.
+     */
+    public boolean completeJoinFuture(
+        GenericGroupMember member,
+        JoinGroupResponseData response
+    ) {
+        if (member.isAwaitingJoin()) {
+            member.awaitingJoinFuture().complete(response);
+            member.setAwaitingJoinFuture(null);
+            numMembersAwaitingJoinResponse--;
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Complete a member's sync future.
+     * 
+     * @param member    the member.
+     * @param response  the sync response to complete the future with.
+     * @return true if a sync future actually completes.
+     */
+    public boolean completeSyncFuture(
+        GenericGroupMember member,
+        SyncGroupResponseData response
+    ) {
+        if (member.isAwaitingSync()) {
+            member.awaitingSyncFuture().complete(response);
+            member.setAwaitingSyncFuture(null);
+            numMembersAwaitingJoinResponse--;

Review Comment:
   I think that this is incorrect, isn't it?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190339738


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.

Review Comment:
   nit: true if the states match, false otherwise



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191602126


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;

Review Comment:
   This is a bit awkward as the existing GroupMetadata updates this field when we read the group metadata record (https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala#L1225). 
   
   So we should expect the new group metadata manager introduced in https://github.com/apache/kafka/pull/13639 to perform this. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190204060


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;

Review Comment:
   it's only used to initializer state. removed it



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190199871


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,876 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Stable;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", Empty, Time.SYSTEM);

Review Comment:
   i'll keep this as is since this variable is used for all tests.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,876 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Stable;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", Empty, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PreparingRebalance);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Empty);
+        group.transitionTo(Dead);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PreparingRebalance);
+        assertState(group, PreparingRebalance);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        group.transitionTo(PreparingRebalance);
+        assertState(group, PreparingRebalance);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Empty);
+        assertState(group, Empty);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Empty);
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        group.transitionTo(Stable);
+        assertState(group, Stable);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        group.transitionTo(Stable);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(CompletingRebalance));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PreparingRebalance));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(CompletingRebalance));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PreparingRebalance));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(CompletingRebalance));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by public voidault, the group supports everything

Review Comment:
   thanks for the catch!



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191303709


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }

Review Comment:
   got it. this is from
   `def isConsumerGroup: Boolean = protocolType.contains(ConsumerProtocol.PROTOCOL_TYPE)`
   
   the naming is confusing because we both consumer and generic groups but a generic group can expect a group using the consumer protocol



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190181383


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {

Review Comment:
   yeah, i agree. i guess "current" was there to emphasize that there could be multiple (older) member ids for a group instance id but i think that is implied as you have mentioned.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1188744749


##########
core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala:
##########
@@ -160,6 +160,7 @@ class GroupMetadataTest {
     assertThrows(classOf[IllegalStateException], () => group.transitionTo(CompletingRebalance))
   }
 
+  @Test

Review Comment:
   Nice catch! Do you mind doing a separate PR for this one? This way we can keep changes in this PR scoped in the new module.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190306648


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.

Review Comment:
   nit: The list of topics that* the group members are subscribed to.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190358529


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all member's rebalance timeout.
+     */
+    public int rebalanceTimeoutMs() {
+        int maxRebalanceTimeoutMs = 0;
+        for (GenericGroupMember member : members.values()) {
+            maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
+        }
+        return maxRebalanceTimeoutMs;
+    }
+
+    /**
+     * Generate a member id from the given client and group instance ids.
+     *
+     * @param clientId the client id.
+     * @param groupInstanceId the group instance id.
+     * @return the generated id.
+     */

Review Comment:
   Should we indent this better? 
        * @param clientId        the client id.
        * @param groupInstanceId the group instance id.
        * @return                the generated id.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190333378


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.

Review Comment:
   nit: Map of protocol names to the number of members that support them.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191340188


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     * @param future  the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, choose a
+     * new leader from one of the joined members.
+     *
+     * Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;

Review Comment:
   thanks for the catch. added a test case, `testMaybeElectNewJoinedLeaderChooseExisting`



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191602652


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,1012 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members awaiting a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to the number of members that support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics to which the group members are subscribed.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * The type of this group.
+     *
+     * @return The group type (Generic).
+     */
+    @Override
+    public GroupType type() {
+        return GroupType.GENERIC;
+    }
+
+    /**
+     * The state of this group.
+     *
+     * @return The current state as a String.
+     */
+    @Override
+    public String stateAsString() {
+        return this.state.toString();
+    }
+
+    /**
+     * @return the group id.
+     */
+    public String groupId() {
+        return this.groupId;
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the protocol type.
+     */
+    public Optional<String> protocolType() {
+        return this.protocolType;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if the state matches, false otherwise.
+     */
+    public boolean isInState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata associated with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean usesConsumerGroupProtocol() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     */
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member  the member to add.
+     * @param future  the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(member)) {
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, choose a
+     * new leader from one of the joined members.
+     *
+     * Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return true if a new leader was elected or the existing
+     *         leader rejoined, false otherwise.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                                "failed to join before the rebalance timeout. Member {} " +
+                                "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId  the group instance id.
+     * @param oldMemberId      the old member id.
+     * @param newMemberId      the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaitingJoinResponse() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all members' rebalance timeout.
+     */
+    public int rebalanceTimeoutMs() {
+        int maxRebalanceTimeoutMs = 0;
+        for (GenericGroupMember member : members.values()) {
+            maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
+        }
+        return maxRebalanceTimeoutMs;
+    }
+
+    /**
+     * Generate a member id from the given client and group instance ids.
+     *
+     * @param clientId         the client id.
+     * @param groupInstanceId  the group instance id.
+     * @return the generated id.
+     */
+    public String generateMemberId(String clientId, Optional<String> groupInstanceId) {
+        return groupInstanceId.map(s -> s + MEMBER_ID_DELIMITER + UUID.randomUUID())
+            .orElseGet(() -> clientId + MEMBER_ID_DELIMITER + UUID.randomUUID());
+    }
+
+    /**
+     * Verify the member id is up to date for static members. Return true if both conditions met:
+     *   1. given member is a known static member to group
+     *   2. group stored member id doesn't match with given member id
+     *
+     * @param groupInstanceId  the group instance id.
+     * @param memberId         the member id.
+     * @return whether the static member is fenced based on the condition above.
+     */
+    public boolean isStaticMemberFenced(
+        String groupInstanceId,
+        String memberId
+    ) {
+        String existingMemberId = staticMemberId(groupInstanceId);
+        return existingMemberId != null && !existingMemberId.equals(memberId);
+    }
+
+    /**
+     * @return whether the group can rebalance.
+     */
+    public boolean canRebalance() {
+        return PREPARING_REBALANCE.validPreviousStates().contains(state);
+    }
+
+    /**
+     * Transition to a group state.
+     * @param groupState the group state.
+     */
+    public void transitionTo(GenericGroupState groupState) {
+        assertValidTransition(groupState);
+        state = groupState;
+        currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * Select a protocol that will be used for this group. Each member
+     * will vote for a protocol and the one with the most votes will
+     * be selected. Only a protocol that is supported by all members
+     * can be selected.
+     *
+     * @return the name of the selected protocol.
+     */
+    public String selectProtocol() {
+        if (members.isEmpty()) {
+            throw new IllegalStateException("Cannot select protocol for empty group");
+        }
+
+        // select the protocol for this group which is supported by all members
+        Set<String> candidates = candidateProtocols();
+
+        // let each member vote for one of the protocols
+        Map<String, Integer> votesByProtocol = new HashMap<>();
+        allMembers().stream().map(member -> member.vote(candidates))
+            .forEach(protocolName -> {
+                int count = votesByProtocol.getOrDefault(protocolName, 0);
+                votesByProtocol.put(protocolName, count + 1);
+            });
+
+        // choose the one with the most votes
+        return votesByProtocol.entrySet().stream()
+            .max(Comparator.comparingInt(Map.Entry::getValue))
+            .map(Map.Entry::getKey).orElse(null);
+    }
+
+    /**
+     * Increment the protocol count for all of the member's
+     * supported protocols.
+     *
+     * @param member the member.
+     */
+    private void incrementSupportedProtocols(GenericGroupMember member) {
+        member.supportedProtocols().forEach(protocol -> {
+            int count = supportedProtocols.getOrDefault(protocol.name(), 0);
+            supportedProtocols.put(protocol.name(), count + 1);
+        });
+    }
+
+    /**
+     * Decrement the protocol count for all of the member's
+     * supported protocols.
+     *
+     * @param member the member.
+     */
+    private void decrementSupportedProtocols(GenericGroupMember member) {
+        member.supportedProtocols().forEach(protocol -> {
+            int count = supportedProtocols.getOrDefault(protocol.name(), 0);
+            supportedProtocols.put(protocol.name(), count - 1);
+        });
+    }
+
+    /**
+     * A candidate protocol must be supported by all members.
+     *
+     * @return a set of candidate protocols that can be chosen as the protocol
+     *         for the group.
+     */
+    private Set<String> candidateProtocols() {
+        // get the set of protocols that are commonly supported by all members
+        return supportedProtocols.entrySet().stream()
+            .filter(protocol -> protocol.getValue() == members.size())
+            .map(Map.Entry::getKey).collect(Collectors.toSet());
+    }
+
+    /**
+     * Checks whether at least one of the given protocols can be supported. A
+     * protocol can be supported if it is supported by all members.
+     *
+     * @param member               the member to check.
+     * @return a boolean based on the condition mentioned above.
+     */
+    public boolean supportsProtocols(GenericGroupMember member) {
+        return supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols())
+        );
+    }
+
+    /**
+     * Checks whether at least one of the given protocols can be supported. A
+     * protocol can be supported if it is supported by all members.
+     *
+     * @param memberProtocolType  the member protocol type.
+     * @param memberProtocols     the set of protocol names.
+     * @return a boolean based on the condition mentioned above.
+     */
+    public boolean supportsProtocols(String memberProtocolType, Set<String> memberProtocols) {
+        if (isInState(EMPTY)) {
+            return !memberProtocolType.isEmpty() && !memberProtocols.isEmpty();
+        } else {
+            return protocolType.map(type -> type.equals(memberProtocolType)).orElse(false) &&
+                memberProtocols.stream()
+                    .anyMatch(name -> supportedProtocols.getOrDefault(name, 0) == members.size());
+        }
+    }
+
+    /**
+     * @return the topics that the group is subscribed to.
+     */
+    public Optional<Set<String>> subscribedTopics() {
+        return subscribedTopics;
+    }
+
+    /**
+     * Returns true if the consumer group is actively subscribed to the topic. When the consumer
+     * group does not know, because the information is not available yet or because the it has
+     * failed to parse the Consumer Protocol, it returns true to be safe.
+     *
+     * @param topic the topic name.
+     * @return whether the group is subscribed to the topic.
+     */
+    public boolean isSubscribedToTopic(String topic) {
+        return subscribedTopics.map(topics -> topics.contains(topic))
+            .orElse(true);
+    }
+
+    /**
+     * Collects the set of topics that the members are subscribed to when the Protocol Type is equal
+     * to 'consumer'. None is returned if
+     * - the protocol type is not equal to 'consumer';
+     * - the protocol is not defined yet; or
+     * - the protocol metadata does not comply with the schema.
+     *
+     * @return the subscribed topics or None based on the condition above.
+     */
+    Optional<Set<String>> computeSubscribedTopics() {
+        if (!protocolType.isPresent()) {
+            return Optional.empty();
+        }
+        String type = protocolType.get();
+        if (!type.equals(ConsumerProtocol.PROTOCOL_TYPE)) {
+            return Optional.empty();
+        }
+        if (members.isEmpty()) {
+            return Optional.of(Collections.emptySet());
+        }
+
+        if (protocolName.isPresent()) {
+            try {
+                Set<String> allSubscribedTopics = new HashSet<>();
+                members.values().forEach(member -> {
+                    ByteBuffer buffer = ByteBuffer.wrap(member.metadata(protocolName.get()));
+                    ConsumerProtocol.deserializeVersion(buffer);
+                    allSubscribedTopics.addAll(new HashSet<>(
+                        ConsumerProtocol.deserializeSubscription(buffer, (short) 0).topics()
+                    ));
+                });
+                return Optional.of(allSubscribedTopics);
+            } catch (SchemaException e) {
+                log.warn("Failed to parse Consumer Protocol " + ConsumerProtocol.PROTOCOL_TYPE + ":" +
+                    protocolName.get() + " of group " + groupId + ". " +
+                    "Consumer group coordinator is not aware of the subscribed topics.", e);
+            }
+        }
+
+        return Optional.empty();
+    }
+
+    /**
+     * Update a member.
+     *
+     * @param member              the member.
+     * @param protocols           the list of protocols.
+     * @param rebalanceTimeoutMs  the rebalance timeout in milliseconds.
+     * @param sessionTimeoutMs    the session timeout in milliseconds.
+     * @param future              the future that is invoked once the join phase is complete.
+     */
+    public void updateMember(
+        GenericGroupMember member,
+        List<Protocol> protocols,
+        int rebalanceTimeoutMs,
+        int sessionTimeoutMs,
+        CompletableFuture<JoinGroupResponseData> future
+    ) {
+        decrementSupportedProtocols(member);
+        member.setSupportedProtocols(protocols);
+        incrementSupportedProtocols(member);
+        member.setRebalanceTimeoutMs(rebalanceTimeoutMs);
+        member.setSessionTimeoutMs(sessionTimeoutMs);
+
+        if (future != null && !member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        } else if (future == null && member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse--;
+        }
+        member.setAwaitingJoinFuture(future);
+    }
+
+    /**
+     * Complete the join future.
+     * 
+     * @param member    the member.
+     * @param response  the join response to complete the future with.
+     * @return true if a join future actually completes.
+     */
+    public boolean completeJoinFuture(
+        GenericGroupMember member,
+        JoinGroupResponseData response
+    ) {
+        if (member.isAwaitingJoin()) {
+            member.awaitingJoinFuture().complete(response);
+            member.setAwaitingJoinFuture(null);
+            numMembersAwaitingJoinResponse--;
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Complete a member's sync future.
+     * 
+     * @param member    the member.
+     * @param response  the sync response to complete the future with.
+     * @return true if a sync future actually completes.
+     */
+    public boolean completeSyncFuture(
+        GenericGroupMember member,
+        SyncGroupResponseData response
+    ) {
+        if (member.isAwaitingSync()) {
+            member.awaitingSyncFuture().complete(response);
+            member.setAwaitingSyncFuture(null);
+            numMembersAwaitingJoinResponse--;

Review Comment:
   thanks for the catch! added a check in `testCompleteSyncFuture`



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190154104


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {

Review Comment:
   that makes sense



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190315524


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupState.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.generic;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Represents all states that a generic group can be in, as well as the states that a group must
+ * be in to transition to a particular state.
+ */
+public enum GenericGroupState {
+
+    /**
+     * Group has no more members, but lingers until all offsets have expired. This state
+     * also represents groups which use Kafka only for offset commits and have no members.
+     *
+     * action: respond normally to join group from new members
+     *         respond to sync group with UNKNOWN_MEMBER_ID
+     *         respond to heartbeat with UNKNOWN_MEMBER_ID
+     *         respond to leave group with UNKNOWN_MEMBER_ID
+     *         respond to offset commit with UNKNOWN_MEMBER_ID
+     *         allow offset fetch requests
+     * transition: last offsets removed in periodic expiration task => DEAD
+     *             join group from a new member => PREPARING_REBALANCE
+     *             group is removed by partition emigration => DEAD
+     *             group is removed by expiration => DEAD
+     */
+    EMPTY(),
+
+    /**
+     * Group is preparing to rebalance

Review Comment:
   nit: missing a full stop



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190294950


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.
+ */
+

Review Comment:
   nit: we can remove this extra line



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190175577


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;

Review Comment:
   thanks for the catch. added a test case



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190330383


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned

Review Comment:
   nit:- of -> from*



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1189445760


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();

Review Comment:
   We don't need the lock in the new runtime.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;

Review Comment:
   Do we need to store this or is it only used to initialize `state`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();

Review Comment:
   Should we call it `pendingJoinMembers` to be consistent with `pendingSyncMembers`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {

Review Comment:
   nit: javadoc?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {

Review Comment:
   The usage of `current` is interesting here. It is somewhat implied so I wonder if `staticMemberId` could work as well. What do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingMembers.isEmpty();
+    }
+
+    /**
+     * @return all members in the group.
+     */
+    public Set<String> allMembers() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMembers() {

Review Comment:
   nit: `allStaticMemberIds`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupState.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.generic;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupState} that is used
+ * by the new group coordinator (KIP-848).

Review Comment:
   Let's update this.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,876 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Stable;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", Empty, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PreparingRebalance);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Empty);
+        group.transitionTo(Dead);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PreparingRebalance);
+        assertState(group, PreparingRebalance);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        group.transitionTo(PreparingRebalance);
+        assertState(group, PreparingRebalance);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Empty);
+        assertState(group, Empty);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Empty);
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        group.transitionTo(Stable);
+        assertState(group, Stable);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        group.transitionTo(Stable);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(CompletingRebalance));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PreparingRebalance));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(CompletingRebalance);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(CompletingRebalance));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        group.transitionTo(Dead);
+        assertState(group, Dead);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(Stable));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PreparingRebalance));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PreparingRebalance);
+        group.transitionTo(Dead);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(CompletingRebalance));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by public voidault, the group supports everything

Review Comment:
   `voidault`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.

Review Comment:
   Let's remove this.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,876 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Stable;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", Empty, Time.SYSTEM);

Review Comment:
   nit: I usually prefer to avoid global variable like this in tests but I leave it up to you.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {

Review Comment:
   nit: I always found `is` weird. Should we use `inState` or something like this?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,

Review Comment:
   nit: Should we put `newMemberId` on a new line as well?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {

Review Comment:
   nit: `hasMember`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));

Review Comment:
   It may be better to throw `IllegalStateException` instead of using `assert`.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;

Review Comment:
   Does this need to be outside of the for loop? Let's extend tests if it is the case.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingMembers.isEmpty();
+    }
+
+    /**
+     * @return all members in the group.
+     */
+    public Set<String> allMembers() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMembers() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMembers() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMembers());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingMembers.size();
+    }

Review Comment:
   nit: Should we move this one to the others "pending members" methods?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {

Review Comment:
   nit: `member` may be better than `get`. what do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupState.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.generic;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupState} that is used
+ * by the new group coordinator (KIP-848).
+ */
+public enum GenericGroupState {
+
+    /**
+     * Group has no more members, but lingers until all offsets have expired. This state
+     * also represents groups which use Kafka only for offset commits and have no members.
+     *
+     * action: respond normally to join group from new members
+     *         respond to sync group with UNKNOWN_MEMBER_ID
+     *         respond to heartbeat with UNKNOWN_MEMBER_ID
+     *         respond to leave group with UNKNOWN_MEMBER_ID
+     *         respond to offset commit with UNKNOWN_MEMBER_ID
+     *         allow offset fetch requests
+     * transition: last offsets removed in periodic expiration task => Dead
+     *             join group from a new member => PreparingRebalance
+     *             group is removed by partition emigration => Dead
+     *             group is removed by expiration => Dead
+     */
+    Empty(),

Review Comment:
   I think that we usually use capital letters (like constants) for enums.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingMembers.isEmpty();
+    }
+
+    /**
+     * @return all members in the group.
+     */
+    public Set<String> allMembers() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMembers() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMembers() {

Review Comment:
   nit: `allDynamicMemberIds`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingMembers.isEmpty();
+    }
+
+    /**
+     * @return all members in the group.
+     */
+    public Set<String> allMembers() {

Review Comment:
   nit: `allMemberIds`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingMembers.isEmpty();
+    }
+
+    /**
+     * @return all members in the group.
+     */
+    public Set<String> allMembers() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMembers() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMembers() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMembers());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingMembers.size();
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allGenericGroupMembers() {

Review Comment:
   nit: `allMembers`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        assert Objects.equals(this.protocolType.orElse(null), member.protocolType());
+        assert supportsProtocols(member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()));
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = get(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                    log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                        currentLeader.memberId(),
+                        currentLeader.groupInstanceId().orElse("None")
+                    );
+                    return false;
+                }
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingMembers.add(memberId);
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!has(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String currentStaticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingMembers.isEmpty();
+    }
+
+    /**
+     * @return all members in the group.
+     */
+    public Set<String> allMembers() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMembers() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMembers() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMembers());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingMembers.size();
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allGenericGroupMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all member's rebalance timeout.
+     */
+    public int rebalanceTimeoutMs() {
+        int maxRebalanceTimeoutMs = 0;
+        for (GenericGroupMember member : members.values()) {
+            maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
+        }
+        return maxRebalanceTimeoutMs;
+    }
+
+    /**
+     * Generate a member id from the given client and group instance ids.
+     *
+     * @param clientId the client id.
+     * @param groupInstanceId the group instance id.
+     * @return the generated id.
+     */
+    public String generateMemberId(String clientId, Optional<String> groupInstanceId) {
+        return groupInstanceId.map(s -> s + MEMBER_ID_DELIMITER + UUID.randomUUID())
+            .orElseGet(() -> clientId + MEMBER_ID_DELIMITER + UUID.randomUUID());
+    }
+
+    /**
+     * Verify the member id is up to date for static members. Return true if both conditions met:
+     *   1. given member is a known static member to group
+     *   2. group stored member id doesn't match with given member id
+     *
+     * @param groupInstanceId the group instance id.
+     * @param memberId the member id.
+     * @return whether the static member is fenced based on the condition above.
+     */
+    public boolean isStaticMemberFenced(
+        String groupInstanceId,
+        String memberId
+    ) {
+        String existingMemberId = currentStaticMemberId(groupInstanceId);
+        return existingMemberId != null && !existingMemberId.equals(memberId);
+    }
+
+    /**
+     * @return whether the group can rebalance.
+     */
+    public boolean canRebalance() {
+        return PreparingRebalance.validPreviousStates().contains(state);
+    }
+
+    /**
+     * Transition to a group state.
+     * @param groupState the group state.
+     */
+    public void transitionTo(GenericGroupState groupState) {
+        assertValidTransition(groupState);
+        state = groupState;
+        currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * Select a protocol that will be used for this group. Each member
+     * will vote for a protocol and the one with the most votes will
+     * be selected. Only a protocol that is supported by all members
+     * can be selected.
+     *
+     * @return the name of the selected protocol.
+     */
+    public String selectProtocol() {
+        if (members.isEmpty()) {
+            throw new IllegalStateException("Cannot select protocol for empty group");
+        }
+
+        // select the protocol for this group which is supported by all members
+        Set<String> candidates = candidateProtocols();
+
+        // let each member vote for one of the protocols
+        Map<String, Integer> votesByProtocol = new HashMap<>();
+        allGenericGroupMembers().stream().map(member -> member.vote(candidates))
+            .forEach(protocolName -> {
+                int count = votesByProtocol.getOrDefault(protocolName, 0);
+                votesByProtocol.put(protocolName, count + 1);
+            });
+
+        // choose the one with the most votes
+        return votesByProtocol.entrySet().stream()
+            .max(Comparator.comparingInt(Map.Entry::getValue))
+            .map(Map.Entry::getKey).orElse(null);
+    }
+
+    /**
+     * Increment the protocol count for all of the member's
+     * supported protocols.
+     *
+     * @param member the member.
+     */
+    private void incrementSupportedProtocols(GenericGroupMember member) {
+        member.supportedProtocols().forEach(protocol -> {
+            int count = supportedProtocols.getOrDefault(protocol.name(), 0);
+            supportedProtocols.put(protocol.name(), count + 1);
+        });
+    }
+
+    /**
+     * Decrement the protocol count for all of the member's
+     * supported protocols.
+     *
+     * @param member the member.
+     */
+    private void decrementSupportedProtocols(GenericGroupMember member) {
+        member.supportedProtocols().forEach(protocol -> {
+            int count = supportedProtocols.getOrDefault(protocol.name(), 0);
+            supportedProtocols.put(protocol.name(), count - 1);
+        });
+    }
+
+    /**
+     * A candidate protocol must be supported by all members.
+     *
+     * @return a set of candidate protocols that can be chosen as the protocol
+     *         for the group.
+     */
+    private Set<String> candidateProtocols() {
+        // get the set of protocols that are commonly supported by all members
+        return supportedProtocols.entrySet().stream()
+            .filter(protocol -> protocol.getValue() == members.size())
+            .map(Map.Entry::getKey).collect(Collectors.toSet());
+    }
+
+    /**
+     * Checks whether at least one of the given protocols can be supported. A
+     * protocol can be supported if it is supported by all members.
+     *
+     * @param memberProtocolType the member protocol type.
+     * @param memberProtocols the set of protocol names.
+     * @return a boolean based on the condition mentioned above.
+     */
+    public boolean supportsProtocols(String memberProtocolType, Set<String> memberProtocols) {
+        if (is(Empty)) {
+            return !memberProtocolType.isEmpty() && !memberProtocols.isEmpty();
+        } else {
+            return protocolType.map(type -> type.equals(memberProtocolType))
+                .orElse(false) &&

Review Comment:
   nit: `orElse` reads a bit weird here. Should we put it on the previous line?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190519534


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group

Review Comment:
   this looks grammatically correct



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190517157


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned

Review Comment:
   this looks grammatically correct



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190312512


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be

Review Comment:
   nit: Check whether the* current leader has rejoined. If not, try to find a new leader among the other joined members.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190356950


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.state = Objects.requireNonNull(initialState);
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean isState(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member id is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean hasMemberId(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember member(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {
+        add(member, null);
+    }
+
+    /**
+     * Add a member to this group.
+     *
+     * @param member the member to add.
+     * @param future the future to complete once the join group phase completes.
+     */
+    public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+        member.groupInstanceId().ifPresent(instanceId -> {
+            if (staticMembers.containsKey(instanceId)) {
+                throw new IllegalStateException("Static member with groupInstanceId=" +
+                    instanceId + " cannot be added to group " + groupId + " since" +
+                    " it is already a member.");
+            }
+            staticMembers.put(instanceId, member.memberId());
+        });
+
+        if (members.isEmpty()) {
+            this.protocolType = Optional.of(member.protocolType());
+        }
+
+        if (!Objects.equals(this.protocolType.orElse(null), member.protocolType())) {
+            throw new IllegalStateException("The group and member's protocol type must be the same.");
+        }
+
+        if (!supportsProtocols(
+            member.protocolType(),
+            GenericGroupMember.plainProtocolSet(member.supportedProtocols()))) {
+
+            throw new IllegalStateException("None of the member's protocols can be supported.");
+        }
+
+        if (!leaderId.isPresent()) {
+            leaderId = Optional.of(member.memberId());
+        }
+
+        members.put(member.memberId(), member);
+        incrementSupportedProtocols(member);
+        member.setAwaitingJoinFuture(future);
+
+        if (member.isAwaitingJoin()) {
+            numMembersAwaitingJoinResponse++;
+        }
+
+        pendingJoinMembers.remove(member.memberId());
+    }
+
+    /**
+     * Remove a member from the group.
+     *
+     * @param memberId the member id to remove.
+     */
+    public void remove(String memberId) {
+        GenericGroupMember removedMember = members.remove(memberId);
+        if (removedMember != null) {
+            decrementSupportedProtocols(removedMember);
+            if (removedMember.isAwaitingJoin()) {
+                numMembersAwaitingJoinResponse--;
+            }
+
+            removedMember.groupInstanceId().ifPresent(staticMembers::remove);
+        }
+
+        if (isLeader(memberId)) {
+            Iterator<String> iter = members.keySet().iterator();
+            String newLeader = iter.hasNext() ? iter.next() : null;
+            leaderId = Optional.ofNullable(newLeader);
+        }
+
+        pendingJoinMembers.remove(memberId);
+        pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * Check whether current leader has rejoined. If not, try to find another joined member to be
+     * new leader. Return false if
+     *   1. the group is currently empty (has no designated leader)
+     *   2. no member rejoined
+     *
+     * @return whether a new leader was elected.
+     */
+    public boolean maybeElectNewJoinedLeader() {
+        if (leaderId.isPresent()) {
+            GenericGroupMember currentLeader = member(leaderId.get());
+            if (!currentLeader.isAwaitingJoin()) {
+                for (GenericGroupMember member : members.values()) {
+                    if (member.isAwaitingJoin()) {
+                        leaderId = Optional.of(member.memberId());
+                        log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                            "failed to join before the rebalance timeout. Member {} " +
+                            "was elected as the new leader.",
+                            currentLeader.memberId(),
+                            currentLeader.groupInstanceId().orElse("None"),
+                            member
+                        );
+                        return true;
+                    }
+                }
+                log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
+                        "failed to join before the rebalance timeout and the " +
+                        "group couldn't proceed to the next generation because " +
+                        "no member joined.",
+                    currentLeader.memberId(),
+                    currentLeader.groupInstanceId().orElse("None")
+                );
+                return false;
+            }
+            return false;
+        }
+        return false;
+    }
+
+    /**
+     * [For static members only]: Replace the old member id with the new one,
+     * keep everything else unchanged and return the updated member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @param oldMemberId the old member id.
+     * @param newMemberId the new member id that will replace the old member id.
+     * @return the old member.
+     */
+    public GenericGroupMember replaceStaticMember(
+        String groupInstanceId,
+        String oldMemberId,
+        String newMemberId
+    ) {
+        GenericGroupMember removedMember = members.remove(oldMemberId);
+        if (removedMember == null) {
+            throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
+        }
+
+        // Fence potential duplicate member immediately if someone awaits join/sync future.
+        JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setMemberId(oldMemberId)
+            .setGenerationId(NO_GENERATION)
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setLeader(NO_LEADER)
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeJoinFuture(removedMember, joinGroupResponse);
+
+        SyncGroupResponseData syncGroupResponse = new SyncGroupResponseData()
+            .setAssignment(new byte[0])
+            .setProtocolName(null)
+            .setProtocolType(null)
+            .setErrorCode(Errors.FENCED_INSTANCE_ID.code());
+        completeSyncFuture(removedMember, syncGroupResponse);
+
+        GenericGroupMember newMember = new GenericGroupMember(
+            newMemberId,
+            removedMember.groupInstanceId(),
+            removedMember.clientId(),
+            removedMember.clientHost(),
+            removedMember.rebalanceTimeoutMs(),
+            removedMember.sessionTimeoutMs(),
+            removedMember.protocolType(),
+            removedMember.supportedProtocols(),
+            removedMember.assignment()
+        );
+
+        members.put(newMemberId, newMember);
+
+        if (isLeader(oldMemberId)) {
+            leaderId = Optional.of(newMemberId);
+        }
+
+        staticMembers.put(groupInstanceId, newMemberId);
+        return removedMember;
+    }
+
+    /**
+     * Check whether a member has joined the group.
+     *
+     * @param memberId the member id.
+     * @return true if the member has yet to join, false otherwise.
+     */
+    public boolean isPendingMember(String memberId) {
+        return pendingJoinMembers.contains(memberId);
+    }
+
+    /**
+     * Add a pending member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending member,
+     *         false otherwise.
+     */
+    public boolean addPendingMember(String memberId) {
+        if (hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingJoinMembers.add(memberId);
+    }
+
+    /**
+     * @return number of members that are pending join.
+     */
+    public int numPending() {
+        return pendingJoinMembers.size();
+    }
+
+    /**
+     * Add a pending sync member.
+     *
+     * @param memberId the member id.
+     * @return true if the group did not already have the pending sync member,
+     *         false otherwise.
+     */
+    public boolean addPendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending sync member " + memberId +
+                " which is already a stable member of the group.");
+        }
+
+        return pendingSyncMembers.add(memberId);
+    }
+
+    /**
+     * Remove a member that has not yet synced.
+     *
+     * @param memberId the member id.
+     * @return true if the group did store this member, false otherwise.
+     */
+    public boolean removePendingSyncMember(String memberId) {
+        if (!hasMemberId(memberId)) {
+            throw new IllegalStateException("Attept to add pending member " + memberId +
+                " which is already a stable member of the group.");
+        }
+        return pendingSyncMembers.remove(memberId);
+    }
+
+    /**
+     * @return true if all members have sent sync group requests,
+     *         false otherwise.
+     */
+    public boolean hasReceivedSyncFromAllMembers() {
+        return pendingSyncMembers.isEmpty();
+    }
+
+    /**
+     * @return members that have yet to sync.
+     */
+    public Set<String> allPendingSyncMembers() {
+        return pendingSyncMembers;
+    }
+
+    /**
+     * Clear members pending sync.
+     */
+    public void clearPendingSyncMembers() {
+        pendingSyncMembers.clear();
+    }
+
+    /**
+     * Checks whether the given group instance id exists as
+     * a static member.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return true if a static member with the group instance id exists,
+     *         false otherwise.
+     */
+    public boolean hasStaticMember(String groupInstanceId) {
+        return staticMembers.containsKey(groupInstanceId);
+    }
+
+    /**
+     * Get member id of a static member that matches the given group
+     * instance id.
+     *
+     * @param groupInstanceId the group instance id.
+     * @return the static member if it exists.
+     */
+    public String staticMemberId(String groupInstanceId) {
+        return staticMembers.get(groupInstanceId);
+    }
+
+    /**
+     * @return members who have yet to rejoin during the
+     *         join group phase.
+     */
+    public Map<String, GenericGroupMember> notYetRejoinedMembers() {
+        Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+        members.values().forEach(member -> {
+            if (!member.isAwaitingJoin()) {
+                notYetRejoinedMembers.put(member.memberId(), member);
+            }
+        });
+        return notYetRejoinedMembers;
+    }
+
+    /**
+     * @return whether all members have joined.
+     */
+    public boolean hasAllMembersJoined() {
+        return members.size() == numMembersAwaitingJoinResponse && pendingJoinMembers.isEmpty();
+    }
+
+    /**
+     * @return the ids of all members in the group.
+     */
+    public Set<String> allMemberIds() {
+        return members.keySet();
+    }
+
+    /**
+     * @return all static members in the group.
+     */
+    public Set<String> allStaticMemberIds() {
+        return staticMembers.keySet();
+    }
+
+    // For testing only.
+    Set<String> allDynamicMemberIds() {
+        Set<String> dynamicMemberSet = new HashSet<>(allMemberIds());
+        staticMembers.values().forEach(dynamicMemberSet::remove);
+        return dynamicMemberSet;
+    }
+
+    /**
+     * @return number of members waiting for a join group response.
+     */
+    public int numAwaiting() {
+        return numMembersAwaitingJoinResponse;
+    }
+
+    /**
+     * @return all members.
+     */
+    public Collection<GenericGroupMember> allMembers() {
+        return members.values();
+    }
+
+    /**
+     * @return the group's rebalance timeout in milliseconds.
+     *         It is the max of all member's rebalance timeout.

Review Comment:
   nit: member's -> members'



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1192103413


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;

Review Comment:
   @clolov Your understanding is correct. The scala code doing this is [here](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala#L1225). We will need to add a setter at some point.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190316117


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupState.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.generic;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Represents all states that a generic group can be in, as well as the states that a group must
+ * be in to transition to a particular state.
+ */
+public enum GenericGroupState {
+
+    /**
+     * Group has no more members, but lingers until all offsets have expired. This state
+     * also represents groups which use Kafka only for offset commits and have no members.
+     *
+     * action: respond normally to join group from new members
+     *         respond to sync group with UNKNOWN_MEMBER_ID
+     *         respond to heartbeat with UNKNOWN_MEMBER_ID
+     *         respond to leave group with UNKNOWN_MEMBER_ID
+     *         respond to offset commit with UNKNOWN_MEMBER_ID
+     *         allow offset fetch requests
+     * transition: last offsets removed in periodic expiration task => DEAD
+     *             join group from a new member => PREPARING_REBALANCE
+     *             group is removed by partition emigration => DEAD
+     *             group is removed by expiration => DEAD
+     */
+    EMPTY(),
+
+    /**
+     * Group is preparing to rebalance

Review Comment:
   same with most of the other comments



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191374954


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java:
##########
@@ -0,0 +1,935 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericGroupTest {
+    private final String protocolType = "consumer";
+    private final String groupInstanceId = "groupInstanceId";
+    private final String memberId = "memberId";
+    private final String clientId = "clientId";
+    private final String clientHost = "clientHost";
+    private final int rebalanceTimeoutMs = 60000;
+    private final int sessionTimeoutMs = 10000;
+
+    
+    private GenericGroup group = null;
+    
+    @BeforeEach
+    public void initialize() {
+        group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM);
+    }
+    
+    @Test
+    public void testCanRebalanceWhenStable() {
+        assertTrue(group.canRebalance());
+    }
+    
+    @Test
+    public void testCanRebalanceWhenCompletingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertTrue(group.canRebalance()); 
+    }
+    
+    @Test
+    public void testCannotRebalanceWhenPreparingRebalance() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testCannotRebalanceWhenDead() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertFalse(group.canRebalance());
+    }
+
+    @Test
+    public void testStableToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testStableToDeadTransition() {
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToPreparingRebalanceTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(PREPARING_REBALANCE);
+        assertState(group, PREPARING_REBALANCE);
+    }
+
+    @Test
+    public void testPreparingRebalanceToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testPreparingRebalanceToEmptyTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        assertState(group, EMPTY);
+    }
+
+    @Test
+    public void testEmptyToDeadTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testAwaitingRebalanceToStableTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertState(group, STABLE);
+    }
+
+    @Test
+    public void testEmptyToStableIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testStableToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        group.transitionTo(STABLE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testEmptyToAwaitingRebalanceIllegalTransition() {
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testPreparingRebalanceToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testAwaitingRebalanceToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(COMPLETING_REBALANCE);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToDeadIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        group.transitionTo(DEAD);
+        assertState(group, DEAD);
+    }
+
+    @Test
+    public void testDeadToStableIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(STABLE));
+    }
+
+    @Test
+    public void testDeadToPreparingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(PREPARING_REBALANCE));
+    }
+
+    @Test
+    public void testDeadToAwaitingRebalanceIllegalTransition() {
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(DEAD);
+        assertThrows(IllegalStateException.class, () -> group.transitionTo(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testSelectProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("range", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        // now could be either range or robin since there is no majority preference
+        assertTrue(group.selectProtocol().equals("range") ||
+            group.selectProtocol().equals("roundrobin"));
+
+        GenericGroupMember member3 = new GenericGroupMember(
+            "member3",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member3);
+
+        // now we should prefer 'roundrobin'
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolRaisesIfNoMembers() {
+        assertThrows(IllegalStateException.class, () -> group.selectProtocol());
+    }
+
+    @Test
+    public void testSelectProtocolChoosesCompatibleProtocol() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+        group.add(member1);
+
+        List<Protocol> member2Protocols = Arrays.asList(
+            new Protocol("roundrobin", new byte[0]),
+            new Protocol("blah", new byte[0])
+        );
+
+        GenericGroupMember member2 = new GenericGroupMember(
+            "member2",
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member2Protocols
+        );
+        group.add(member2);
+
+        assertEquals("roundrobin", group.selectProtocol());
+    }
+
+    @Test
+    public void testSupportsProtocols() {
+        List<Protocol> member1Protocols = Arrays.asList(
+            new Protocol("range", new byte[0]),
+            new Protocol("roundrobin", new byte[0])
+        );
+
+        GenericGroupMember member1 = new GenericGroupMember(
+            memberId,
+            Optional.empty(),
+            clientId,
+            clientHost,
+            rebalanceTimeoutMs,
+            sessionTimeoutMs,
+            protocolType,
+            member1Protocols
+        );
+
+        // by default, the group supports everything
+        Set<String> expectedProtocols = new HashSet<>();
+        member1Protocols.forEach(protocol -> expectedProtocols.add(protocol.name()));
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));
+
+        group.add(member1);
+        group.transitionTo(PREPARING_REBALANCE);
+
+        expectedProtocols.clear();
+        expectedProtocols.add("roundrobin");
+        expectedProtocols.add("foo");
+        assertTrue(group.supportsProtocols(protocolType, expectedProtocols));

Review Comment:
   thanks for the suggestion!



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190331659


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group

Review Comment:
   nit: have -> are*



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190329439


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.

Review Comment:
   nit: a* client id or a* group instance id



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] clolov commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "clolov (via GitHub)" <gi...@apache.org>.
clolov commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191430711


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,998 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.group.Group;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group where the
+ * member assignment is driven solely from the client side.
+ *
+ * The APIs members use to make changes to the group membership
+ * consist of JoinGroup, SyncGroup, and LeaveGroup.
+ */
+public class GenericGroup implements Group {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with a client id or a group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;

Review Comment:
   Is this necessary to be an Optional? As far as I see we immediately define it in the constructor and we never set it equal to null. Am I missing something obvious?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191678769


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group

Review Comment:
   oya yep mb



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1191160971


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,967 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.CompletingRebalance;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Dead;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.Empty;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PreparingRebalance;
+
+/**
+ * Java rewrite of {@link kafka.coordinator.group.GroupMetadata} that is used
+ * by the new group coordinator (KIP-848). Offset management will be handled
+ * by a different component.
+ *
+ * This class holds group metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The initial group state.
+     */
+    private final GenericGroupState initialState;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The lock used to synchronize the group.
+     */
+    private final Lock lock = new ReentrantLock();
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.
+     */
+    private int numMembersAwaitingJoinResponse = 0;
+
+    /**
+     * Map of protocol names to how many members support them.
+     */
+    private final Map<String, Integer> supportedProtocols = new HashMap<>();
+
+    /**
+     * Members who have yet to sync with the group
+     * during the sync group phase.
+     */
+    private final Set<String> pendingSyncMembers = new HashSet<>();
+
+    /**
+     * The list of topics the group members are subscribed to.
+     */
+    private Optional<Set<String>> subscribedTopics = Optional.empty();
+
+    /**
+     * A flag to indiciate whether a new member was added. Used
+     * to further delay initial joins (new group).
+     */
+    private boolean newMemberAdded = false;
+
+    public GenericGroup(
+        LogContext logContext,
+        String groupId,
+        GenericGroupState initialState,
+        Time time
+    ) {
+        this.logContext = Objects.requireNonNull(logContext);
+        this.log = logContext.logger(GenericGroup.class);
+        this.groupId = Objects.requireNonNull(groupId);
+        this.initialState = Objects.requireNonNull(initialState);
+        this.state = initialState;
+        this.time = Objects.requireNonNull(time);
+        this.currentStateTimestamp = Optional.of(time.milliseconds());
+    }
+
+    /**
+     * @return the generation id.
+     */
+    public int generationId() {
+        return this.generationId;
+    }
+
+    /**
+     * @return the protocol name.
+     */
+    public Optional<String> protocolName() {
+        return this.protocolName;
+    }
+
+    /**
+     * @return the current group state.
+     */
+    public GenericGroupState currentState() {
+        return state;
+    }
+
+    /**
+     * Compares the group's current state with the given state.
+     *
+     * @param groupState the state to match against.
+     * @return true if matches, false otherwise.
+     */
+    public boolean is(GenericGroupState groupState) {
+        return this.state == groupState;
+    }
+
+    /**
+     * To identify whether the given member is part of this group.
+     *
+     * @param memberId the given member id.
+     * @return true if the member is part of this group, false otherwise.
+     */
+    public boolean has(String memberId) {
+        return members.containsKey(memberId);
+    }
+
+    /**
+     * Get the member metadata with the provided member id.
+     *
+     * @param memberId the member id.
+     * @return the member metadata if it exists, null otherwise.
+     */
+    public GenericGroupMember get(String memberId) {
+        return members.get(memberId);
+    }
+
+    /**
+     * @return the total number of members in this group.
+     */
+    public int size() {
+        return members.size();
+    }
+
+    /**
+     * Used to identify whether the given member is the leader of this group.
+     *
+     * @param memberId the member id.
+     * @return true if the member is the leader, false otherwise.
+     */
+    public boolean isLeader(String memberId) {
+        return leaderId.map(id -> id.equals(memberId)).orElse(false);
+    }
+
+    /**
+     * @return the leader id or null if a leader does not exist.
+     */
+    public String leaderOrNull() {
+        return leaderId.orElse(null);
+    }
+
+    /**
+     * @return the current state timestamp.
+     */
+    public long currentStateTimestampOrDefault() {
+        return currentStateTimestamp.orElse(-1L);
+    }
+
+    /**
+     * @return whether the group is using the consumer protocol.
+     */
+    public boolean isGenericGroup() {
+        return protocolType.map(type ->
+            type.equals(ConsumerProtocol.PROTOCOL_TYPE)
+        ).orElse(false);
+    }
+
+    public void add(GenericGroupMember member) {

Review Comment:
   yeah, it is better. that method looks weird otherwise.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13663: KAFKA-14500; [2/N] Rewrite GroupMetadata in Java

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13663:
URL: https://github.com/apache/kafka/pull/13663#discussion_r1190300876


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -0,0 +1,958 @@
+/*
+ * 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.generic;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
+import org.apache.kafka.common.message.JoinGroupResponseData;
+import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.common.message.SyncGroupResponseData;
+import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.slf4j.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
+import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
+
+/**
+ *
+ * This class holds metadata for a generic group.
+ */
+public class GenericGroup {
+
+    /**
+     * Empty generation.
+     */
+    public static final int NO_GENERATION = -1;
+
+    /**
+     * Protocol with empty name.
+     */
+    public static final String NO_PROTOCOL_NAME = "";
+
+    /**
+     * No leader.
+     */
+    public static final String NO_LEADER = "";
+
+    /**
+     * Delimiter used to join a randomly generated UUID
+     * with client id or group instance id.
+     */
+    private static final String MEMBER_ID_DELIMITER = "-";
+
+    /**
+     * The slf4j log context, used to create new loggers.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The slf4j logger.
+     */
+    private final Logger log;
+
+    /**
+     * The group id.
+     */
+    private final String groupId;
+
+    /**
+     * The time.
+     */
+    private final Time time;
+
+    /**
+     * The current group state.
+     */
+    private GenericGroupState state;
+
+    /**
+     * The timestamp of when the group transitioned
+     * to its current state.
+     */
+    private Optional<Long> currentStateTimestamp;
+
+    /**
+     * The protocol type used for rebalance.
+     */
+    private Optional<String> protocolType = Optional.empty();
+
+    /**
+     * The protocol name used for rebalance.
+     */
+    private Optional<String> protocolName = Optional.empty();
+
+    /**
+     * The generation id.
+     */
+    private int generationId = 0;
+
+    /**
+     * The id of the group's leader.
+     */
+    private Optional<String> leaderId = Optional.empty();
+
+    /**
+     * The members of the group.
+     */
+    private final Map<String, GenericGroupMember> members = new HashMap<>();
+
+    /**
+     * The static members of the group.
+     */
+    private final Map<String, String> staticMembers = new HashMap<>();
+
+    /**
+     * Members who have yet to (re)join the group
+     * during the join group phase.
+     */
+    private final Set<String> pendingJoinMembers = new HashSet<>();
+
+    /**
+     * The number of members waiting to hear back a join response.

Review Comment:
   nit: awaiting a join response?* OR waiting for a join response



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org